From 548d79c2e80bb23f246c63fc7e33d0c01eb6b944 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 2 Mar 2023 12:31:09 +0000 Subject: [PATCH 001/871] 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 e36bc470512..00000000000 --- 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/871] 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 291499ff412..64a3a4c74ae 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/871] 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 8c0d50bae55..cd08de126c9 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 57fa75dc67b..ae7e7ab5bf0 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 101f6f1f934..96dbe26f820 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 b8de0246ae2..0a817995eb4 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 23a2860364d..97294d70ead 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 @@ public: 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 @@ private: 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 @@ private: }; 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 @@ private: /// (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 36cb57c3678..f73429913b3 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 dfe2389edb7..9e2dca8bb23 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -72,6 +72,7 @@ public: 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 00000000000..b94888d227e --- /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 00000000000..33f40eef14e --- /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/871] 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 7f1fe838b80..5aa054d43b0 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 d1960860308..93525c3f310 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 @@ public: 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 00000000000..e69de29bb2d 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 00000000000..b8378aee17e --- /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/871] 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 5aa054d43b0..2ab4fe9b32a 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 266d14f645b..33010dc6b3b 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 93525c3f310..db4f0e7dd3e 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -112,7 +112,7 @@ public: 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 @@ public: 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/871] 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 db4f0e7dd3e..fba8293e5ff 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -113,7 +113,7 @@ public: { /// 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 @@ public: 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 ccd9bbcf10e..a31a71ce381 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 e4dad56bc29..e2a23258584 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 4a3d35e48b7..634b19a7624 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 dec1276111a..ac516e83c84 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 9b0fe875416..eb42159ce15 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 ac507d94b69..36bcaae383f 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 2e3080e9cfc..04a10cfe55e 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 12bc3b09472..829352110f6 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 f217b6094b2..d39ffdc4049 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 87e1a039488..32481be1bcd 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 d83656e0e8c..adab3906e5b 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/871] 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 7a51d86dd30..b6ab9b7d0c3 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 eeba62c5dc8..22dfaf93781 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 dc229412b13..939c189c5fe 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/871] 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 33010dc6b3b..e0d23d139f3 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 9032e23f87d2f8ed88ad62c79017c2c951d74464 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 May 2023 15:43:23 +0200 Subject: [PATCH 009/871] Clean up storage_conf.xml, use dynamic disks for tests --- tests/config/config.d/storage_conf.xml | 133 ------------------ ...ilesystem_cache_bypass_cache_threshold.sql | 19 ++- .../02240_filesystem_query_cache.sql | 18 ++- .../02344_describe_cache.reference | 1 - .../0_stateless/02344_describe_cache.sql | 3 - ..._cache_on_write_with_small_segment_size.sh | 17 ++- 6 files changed, 51 insertions(+), 140 deletions(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dee03307177..98798d5746e 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -10,46 +10,6 @@ clickhouse 20000 - - s3 - s3_disk_2/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_3/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_4/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_5/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_6/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - cache @@ -60,54 +20,6 @@ 0 100 - - cache - s3_disk_2 - s3_cache_2/ - 128Mi - 0 - 100Mi - 100 - - - cache - s3_disk_3 - s3_disk_3_cache/ - 128Mi - 22548578304 - 1 - 1 - 0 - 100 - - - cache - s3_disk_4 - s3_cache_4/ - 128Mi - 1 - 1 - 0 - 100 - - - cache - s3_disk_5 - s3_cache_5/ - 128Mi - 0 - 100 - - - cache - s3_disk_6 - s3_cache_6/ - 128Mi - 0 - 1 - 100 - 100 - cache s3_disk_6 @@ -116,16 +28,6 @@ 1 100 - - cache - s3_disk_6 - s3_cache_small_segment_size/ - 128Mi - 10Ki - 0 - 1 - 100 - local_blob_storage @@ -193,34 +95,6 @@ - - -
- s3_cache_2 -
-
-
- - -
- s3_cache_3 -
-
-
- - -
- s3_cache_4 -
-
-
- - -
- s3_cache_6 -
-
-
@@ -256,13 +130,6 @@
- - -
- s3_cache_small_segment_size -
-
-
diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql index f6671b82291..ae2cd1b8cd1 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql @@ -6,7 +6,24 @@ SYSTEM DROP FILESYSTEM CACHE; SET enable_filesystem_cache_on_write_operations=0; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + enable_bypass_cache_with_threashold = 1, + bypass_cache_threashold = 100, + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 94eb4bc5ccd..f2664de5c63 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -8,7 +8,23 @@ SET skip_download_if_exceeds_query_cache=1; SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; + +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7561b32bae1..4302b05e136 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1 @@ 134217728 1048576 104857600 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 diff --git a/tests/queries/0_stateless/02344_describe_cache.sql b/tests/queries/0_stateless/02344_describe_cache.sql index a687ad01394..9c5c5c10952 100644 --- a/tests/queries/0_stateless/02344_describe_cache.sql +++ b/tests/queries/0_stateless/02344_describe_cache.sql @@ -1,7 +1,4 @@ -- Tags: no-fasttest, no-parallel SYSTEM DROP FILESYSTEM CACHE 's3_cache'; -SYSTEM DROP FILESYSTEM CACHE 's3_cache_2'; - DESCRIBE FILESYSTEM CACHE 's3_cache'; -DESCRIBE FILESYSTEM CACHE 's3_cache_2'; diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index ed66c36b823..2c526d10cc9 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -13,7 +13,22 @@ function random { ${CLICKHOUSE_CLIENT} --multiline --multiquery -q " drop table if exists ttt; -create table ttt (id Int32, value String) engine=MergeTree() order by tuple() settings storage_policy='s3_cache_small_segment_size', min_bytes_for_wide_part=0; + +CREATE TABLE ttt (id Int32, value String) +Engine=MergeTree() +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, + disk = disk( + type = cache, + max_size = '128Mi', + max_file_segment_size = '10Ki', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + insert into ttt select number, toString(number) from numbers(100000) settings throw_on_error_from_cache_on_write_operations = 1; " From 431b2e94a936f5021b6d834994614d8d03ddafcb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 5 May 2023 19:10:12 +0200 Subject: [PATCH 010/871] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 98798d5746e..923240d5a91 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -22,7 +22,7 @@ cache - s3_disk_6 + s3_disk s3_cache_small/ 1000 1 From bc16cc59fff3ccab70fd0ca54ece50e15f99e266 Mon Sep 17 00:00:00 2001 From: Bin Xie Date: Sat, 6 May 2023 11:09:45 +0800 Subject: [PATCH 011/871] If a dictionary is created with a complex key, automatically choose the "complex key" layout variant. --- src/Dictionaries/DictionaryFactory.cpp | 23 +++++++++++++++++-- src/Dictionaries/DictionaryFactory.h | 4 +++- src/Dictionaries/FlatDictionary.cpp | 2 +- .../getDictionaryConfigurationFromAST.cpp | 6 +++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index d091e49d1f0..c3102632167 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -17,13 +17,13 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex) +void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex) { auto it = registered_layouts.find(layout_type); if (it != registered_layouts.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type); - RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex }; + RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex, .has_layout_complex = has_layout_complex }; registered_layouts.emplace(layout_type, std::move(layout)); } @@ -89,6 +89,25 @@ bool DictionaryFactory::isComplex(const std::string & layout_type) const return it->second.is_layout_complex; } +bool DictionaryFactory::convertToComplex(std::string & layout_type) const +{ + auto it = registered_layouts.find(layout_type); + + if (it == registered_layouts.end()) + { + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "Unknown dictionary layout type: {}", + layout_type); + } + + if (!it->second.is_layout_complex && it->second.has_layout_complex) + { + layout_type = "complex_key_" + layout_type; + return true; + } + return false; +} + DictionaryFactory & DictionaryFactory::instance() { diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index b1dad340f4b..0dc80af62fc 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -54,14 +54,16 @@ public: bool created_from_ddl)>; bool isComplex(const std::string & layout_type) const; + bool convertToComplex(std::string & layout_type) const; - void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex); + void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex = true); private: struct RegisteredLayout { LayoutCreateFunction layout_create_function; bool is_layout_complex; + bool has_layout_complex; }; using LayoutRegistry = std::unordered_map; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index d3699a150c4..5bbb5a33fa9 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -683,7 +683,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) return std::make_unique(dict_id, dict_struct, std::move(source_ptr), configuration); }; - factory.registerLayout("flat", create_layout, false); + factory.registerLayout("flat", create_layout, false, false); } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0de8b843604..6d3d08a323b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -609,6 +609,12 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type); + if (pk_attrs.size() > 1 && !complex + && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) + { + complex = true; + } + auto all_attr_names_and_types = buildDictionaryAttributesConfiguration( xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); From d3c3d8b8e401d239416f323c69ceb12c67e3c26d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 16:48:40 +0100 Subject: [PATCH 012/871] Remove export of dynamic symbols --- CMakeLists.txt | 10 ++++++++-- programs/library-bridge/CMakeLists.txt | 4 ---- programs/odbc-bridge/CMakeLists.txt | 6 ------ 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 263b202049b..3283ca52ca7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -155,8 +155,14 @@ elseif(GLIBC_COMPATIBILITY) message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") endif () -# Make sure the final executable has symbols exported -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") +if (OS_LINUX) + # We should not export dynamic symbols, because: + # - The main clickhouse binary does not use dlopen, + # and whatever is poisoning it by LD_PRELOAD should not link to our symbols. + # - The clickhouse-odbc-bridge and clickhouse-library-bridge binaries + # should not expose their symbols to ODBC drivers and libraries. + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") +endif () if (OS_DARWIN) # The `-all_load` flag forces loading of all symbols from all libraries, diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 1cacc391ca5..dd0bf67cb64 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,10 +13,6 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) -if (OS_LINUX) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") -endif () - clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 118610e4dcd..56373601b95 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,12 +15,6 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) -if (OS_LINUX) - # clickhouse-odbc-bridge is always a separate binary. - # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") -endif () - clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE From 5a3281bb4912dce6a6125681b0804b97653da763 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 19:30:03 +0100 Subject: [PATCH 013/871] Remove unused code --- src/Common/getResource.cpp | 35 ----------------------------------- 1 file changed, 35 deletions(-) diff --git a/src/Common/getResource.cpp b/src/Common/getResource.cpp index fe603fcc550..09777640dd9 100644 --- a/src/Common/getResource.cpp +++ b/src/Common/getResource.cpp @@ -1,6 +1,4 @@ #include "getResource.h" -#include -#include #include #include @@ -14,39 +12,6 @@ std::string_view getResource(std::string_view name) std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); boost::replace_all(name_replaced, "+", "_PLUS_"); -#if defined USE_MUSL /// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself. return DB::SymbolIndex::instance()->getResource(name_replaced); - -#else - // In most `dlsym(3)` APIs, one passes the symbol name as it appears via - // something like `nm` or `objdump -t`. For example, a symbol `_foo` would be - // looked up with the string `"_foo"`. - // - // Apple's linker is confusingly different. The NOTES on the man page for - // `dlsym(3)` claim that one looks up the symbol with "the name used in C - // source code". In this example, that would mean using the string `"foo"`. - // This apparently applies even in the case where the symbol did not originate - // from C source, such as the embedded binary resource files used here. So - // the symbol name must not have a leading `_` on Apple platforms. It's not - // clear how this applies to other symbols, such as those which _have_ a leading - // underscore in them by design, many leading underscores, etc. -#if defined OS_DARWIN - std::string prefix = "binary_"; -#else - std::string prefix = "_binary_"; -#endif - std::string symbol_name_start = prefix + name_replaced + "_start"; - std::string symbol_name_end = prefix + name_replaced + "_end"; - - const char * sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); - const char * sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); - - if (sym_start && sym_end) - { - auto resource_size = static_cast(std::distance(sym_start, sym_end)); - return { sym_start, resource_size }; - } - return {}; -#endif } From 8cd9fc4a2d6936343c2be119d8fdd61986cdd77f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 21:20:20 +0100 Subject: [PATCH 014/871] Fix build --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 2cd0a011013..c7f31e13287 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -62,7 +62,7 @@ then ninja $NINJA_FLAGS clickhouse-keeper ls -la ./programs/ - ldd ./programs/clickhouse-keeper + ldd ./programs/clickhouse-keeper ||: if [ -n "$MAKE_DEB" ]; then # No quotes because I want it to expand to nothing if empty. From 03845ba9c5fb1ccb03330059369641751f1c9ab1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 00:16:23 +0200 Subject: [PATCH 015/871] Fix MSan. --- src/Common/SymbolIndex.cpp | 81 +++++++++++++++++++++++--------------- 1 file changed, 50 insertions(+), 31 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index f1cace5017c..79f97e93a2f 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -9,7 +9,6 @@ #include -//#include #include #include @@ -63,9 +62,11 @@ Otherwise you will get only exported symbols from program headers. #endif #define __msan_unpoison_string(X) // NOLINT +#define __msan_unpoison(X, Y) // NOLINT #if defined(ch_has_feature) # if ch_has_feature(memory_sanitizer) # undef __msan_unpoison_string +# undef __msan_unpoison # include # endif #endif @@ -136,10 +137,12 @@ void collectSymbolsFromProgramHeaders( /* Iterate over all headers of the current shared lib * (first call is for the executable itself) */ + __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { /* Further processing is only needed if the dynamic section is reached */ + __msan_unpoison(&info->dlpi_phdr[header_index], sizeof(info->dlpi_phdr[header_index])); if (info->dlpi_phdr[header_index].p_type != PT_DYNAMIC) continue; @@ -160,44 +163,53 @@ void collectSymbolsFromProgramHeaders( */ size_t sym_cnt = 0; - for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it) { - ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); - - // TODO: this branch leads to invalid address of the hash table. Need further investigation. - // if (it->d_tag == DT_HASH) - // { - // const ElfW(Word) * hash = reinterpret_cast(base_address); - // sym_cnt = hash[1]; - // break; - // } - if (it->d_tag == DT_GNU_HASH) + const auto * it = dyn_begin; + while (true) { - /// This code based on Musl-libc. + __msan_unpoison(it, sizeof(*it)); + if (it->d_tag != DT_NULL) + break; - const uint32_t * buckets = nullptr; - const uint32_t * hashval = nullptr; + ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); - const ElfW(Word) * hash = reinterpret_cast(base_address); - - buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); - - for (ElfW(Word) i = 0; i < hash[0]; ++i) - if (buckets[i] > sym_cnt) - sym_cnt = buckets[i]; - - if (sym_cnt) + if (it->d_tag == DT_GNU_HASH) { - sym_cnt -= hash[1]; - hashval = buckets + hash[0] + sym_cnt; - do + /// This code based on Musl-libc. + + const uint32_t * buckets = nullptr; + const uint32_t * hashval = nullptr; + + const ElfW(Word) * hash = reinterpret_cast(base_address); + + __msan_unpoison(&hash[0], sizeof(*hash)); + __msan_unpoison(&hash[1], sizeof(*hash)); + __msan_unpoison(&hash[2], sizeof(*hash)); + + buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); + + __msan_unpoison(buckets, hash[0] * sizeof(buckets[0])); + + for (ElfW(Word) i = 0; i < hash[0]; ++i) + if (buckets[i] > sym_cnt) + sym_cnt = buckets[i]; + + if (sym_cnt) { - ++sym_cnt; + sym_cnt -= hash[1]; + hashval = buckets + hash[0] + sym_cnt; + __msan_unpoison(&hashval, sizeof(hashval)); + do + { + ++sym_cnt; + } + while (!(*hashval++ & 1)); } - while (!(*hashval++ & 1)); + + break; } - break; + ++it; } } @@ -228,6 +240,8 @@ void collectSymbolsFromProgramHeaders( /* Get the pointer to the first entry of the symbol table */ const ElfW(Sym) * elf_sym = reinterpret_cast(base_address); + __msan_unpoison(elf_sym, sym_cnt * sizeof(*elf_sym)); + /* Iterate over the symbol table */ for (ElfW(Word) sym_index = 0; sym_index < ElfW(Word)(sym_cnt); ++sym_index) { @@ -235,6 +249,7 @@ void collectSymbolsFromProgramHeaders( * This is located at the address of st_name relative to the beginning of the string table. */ const char * sym_name = &strtab[elf_sym[sym_index].st_name]; + __msan_unpoison_string(sym_name); if (!sym_name) continue; @@ -264,13 +279,17 @@ void collectSymbolsFromProgramHeaders( #if !defined USE_MUSL String getBuildIDFromProgramHeaders(dl_phdr_info * info) { + __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { const ElfPhdr & phdr = info->dlpi_phdr[header_index]; + __msan_unpoison(&phdr, sizeof(phdr)); if (phdr.p_type != PT_NOTE) continue; - return Elf::getBuildID(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz); + std::string_view view(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz); + __msan_unpoison(view.data(), view.size()); + return Elf::getBuildID(view.data(), view.size()); } return {}; } From 9e513a147b5ca7ca0b75feec5488093f32df77d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 04:22:01 +0200 Subject: [PATCH 016/871] Fixup --- src/Common/SymbolIndex.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 79f97e93a2f..394ae1a0592 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -138,6 +138,7 @@ void collectSymbolsFromProgramHeaders( * (first call is for the executable itself) */ __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); + __msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { /* Further processing is only needed if the dynamic section is reached @@ -280,6 +281,7 @@ void collectSymbolsFromProgramHeaders( String getBuildIDFromProgramHeaders(dl_phdr_info * info) { __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); + __msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { const ElfPhdr & phdr = info->dlpi_phdr[header_index]; From 72dd039d1c9cedafea3f6aa6dbf8c47b24ab61c9 Mon Sep 17 00:00:00 2001 From: xbthink Date: Sun, 7 May 2023 16:22:05 +0800 Subject: [PATCH 017/871] add comments and functional test --- src/Dictionaries/DictionaryFactory.h | 3 ++ ...ayout_to_complex_by_complex_keys.reference | 4 +++ ...nary_layout_to_complex_by_complex_keys.sql | 29 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference create mode 100644 tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index 0dc80af62fc..309f14f6b7f 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -54,6 +54,9 @@ public: bool created_from_ddl)>; bool isComplex(const std::string & layout_type) const; + + /// if the argument `layout_type` is not complex layout and has corresponding complex layout, + /// change `layout_type` to corresponding complex and return true; otherwise do nothing and return false. bool convertToComplex(std::string & layout_type) const; void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex = true); diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference new file mode 100644 index 00000000000..7f38556e7d1 --- /dev/null +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -0,0 +1,4 @@ +dict_flat_simple Flat +dict_hashed_simple Hashed +dict_hashed_complex ComplexKeyHashed +dict_hashed_simple_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql new file mode 100644 index 00000000000..98c7c5c05f9 --- /dev/null +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -0,0 +1,29 @@ +DROP DICTIONARY IF EXISTS dict_flat_simple; +DROP DICTIONARY IF EXISTS dict_hashed_simple; +DROP DICTIONARY IF EXISTS dict_hashed_complex; +DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; +DROP TABLE IF EXISTS dict_data; + +CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory() AS SELECT number, number%65535, number%65535, number%6553, number%655355 FROM numbers(100); + +CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); +SYSTEM RELOAD DICTIONARY dict_flat_simple; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_flat_simple'; +DROP DICTIONARY dict_flat_simple; + +CREATE DICTIONARY dict_hashed_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple'; +DROP DICTIONARY dict_hashed_simple; + +CREATE DICTIONARY dict_hashed_complex (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(complex_key_hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_complex; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_complex'; +DROP DICTIONARY dict_hashed_complex; + +CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; +DROP DICTIONARY dict_hashed_simple_auto_convert; + +DROP TABLE dict_data; From 7f9b21849c4ac6dc9243426ae885e3e6b7208d6f Mon Sep 17 00:00:00 2001 From: xiebin Date: Sun, 7 May 2023 19:06:06 +0800 Subject: [PATCH 018/871] Fixed a lowercase initial letter and removed needless data --- src/Dictionaries/DictionaryFactory.h | 2 +- ...uto_convert_dictionary_layout_to_complex_by_complex_keys.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index 309f14f6b7f..35097a5ed24 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -55,7 +55,7 @@ public: bool isComplex(const std::string & layout_type) const; - /// if the argument `layout_type` is not complex layout and has corresponding complex layout, + /// If the argument `layout_type` is not complex layout and has corresponding complex layout, /// change `layout_type` to corresponding complex and return true; otherwise do nothing and return false. bool convertToComplex(std::string & layout_type) const; diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 98c7c5c05f9..0fb06e5acc2 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -4,7 +4,7 @@ DROP DICTIONARY IF EXISTS dict_hashed_complex; DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; DROP TABLE IF EXISTS dict_data; -CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory() AS SELECT number, number%65535, number%65535, number%6553, number%655355 FROM numbers(100); +CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory(); CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); SYSTEM RELOAD DICTIONARY dict_flat_simple; From 8c0b634a644ac85832658cc6ca863909ef455795 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 7 May 2023 14:38:27 +0200 Subject: [PATCH 019/871] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 923240d5a91..aad93a017c2 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -72,7 +72,7 @@ cache - s3_cache_5 + s3_cache s3_cache_multi/ 22548578304 0 From 726222f1ea69018115642156a06c64ec546244d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:33:11 +0200 Subject: [PATCH 020/871] 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 a31a71ce381..926557e4ba6 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 e2a23258584..bcaa70abbb5 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 634b19a7624..decbe136fc4 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 ac516e83c84..6dd7ff3cdc8 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 eb42159ce15..26d3dd8acd4 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 04a10cfe55e..fcdfa2dec82 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 829352110f6..50ade3fad45 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 021/871] 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 860529a26e5..9ed78fd9f81 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 4bd5e79d1b3..30beb29251e 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 0c2dd04d024..b563bc31b39 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 569466fe606..44000e5d2cd 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 022/871] 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 2202a349c56..c1f28f9f079 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 023/871] 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 7a0b9ef8911..7505a03a382 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 0818092ae8d49f2e7f87fed6c8703374384719fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:45:57 +0200 Subject: [PATCH 024/871] Enable Sparse columns by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5416b77a97e..27f482d79ba 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From 7ec98205b58ab36eb28b2f46348dfcfe22215a3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 22:54:14 +0300 Subject: [PATCH 025/871] Update MergeTreeSettings.h --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 27f482d79ba..caac86c6706 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From f3f6ccd7733aa4946c339b4973210f85243e44d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:28:54 +0200 Subject: [PATCH 026/871] Update tests --- .../0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- ...0484_preferred_max_column_in_block_size_bytes.sql | 8 ++++---- .../00804_test_delta_codec_compression.sql | 12 ++++++------ .../0_stateless/00950_test_double_delta_codec.sql | 2 +- ...00961_checksums_in_system_parts_columns_table.sql | 2 +- .../0_stateless/01055_compact_parts_granularity.sh | 2 +- .../queries/0_stateless/01786_explain_merge_tree.sh | 4 ++-- tests/queries/0_stateless/02263_lazy_mark_load.sh | 2 +- .../0_stateless/02293_selected_rows_and_merges.sh | 8 +++----- .../0_stateless/02361_fsync_profile_events.sh | 7 ++++--- .../02381_compress_marks_and_primary_key.sql | 4 ++-- 11 files changed, 28 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index c184b58bf53..27b9f5c00c7 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql index 470bca70e06..be4af2221a5 100644 --- a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql +++ b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql @@ -1,7 +1,7 @@ -- Tags: no-random-settings drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 8192; set preferred_block_size_bytes = 2000000; @@ -17,19 +17,19 @@ set preferred_max_column_in_block_size_bytes = 4194304; select max(blockSize()), min(blockSize()), any(ignore(*)) from tab_00484; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 47; set preferred_max_column_in_block_size_bytes = 1152; select blockSize(), * from tab_00484 where x = 1 or x > 36 format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 10; set preferred_max_column_in_block_size_bytes = 128; select s from tab_00484 where s == '' format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81920; set preferred_block_size_bytes = 0; select count(*) from tab_00484 prewhere s != 'abc' format Null; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 25988f6474b..01a2f53bf93 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -9,12 +9,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -47,12 +47,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -85,12 +85,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00950_test_double_delta_codec.sql b/tests/queries/0_stateless/00950_test_double_delta_codec.sql index f6199a6e4ec..58cf35b5248 100644 --- a/tests/queries/0_stateless/00950_test_double_delta_codec.sql +++ b/tests/queries/0_stateless/00950_test_double_delta_codec.sql @@ -24,7 +24,7 @@ CREATE TABLE codecTest ( valueI8 Int8 CODEC(DoubleDelta), valueDT DateTime CODEC(DoubleDelta), valueD Date CODEC(DoubleDelta) -) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0; +) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; -- checking for overflow diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index 43b7775e816..8df7d728560 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_00961; CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) - SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi'; + SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi', ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); diff --git a/tests/queries/0_stateless/01055_compact_parts_granularity.sh b/tests/queries/0_stateless/01055_compact_parts_granularity.sh index f3da33f6ccf..3e5da1e6f90 100755 --- a/tests/queries/0_stateless/01055_compact_parts_granularity.sh +++ b/tests/queries/0_stateless/01055_compact_parts_granularity.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mt_compact" $CLICKHOUSE_CLIENT -q "CREATE TABLE mt_compact(a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, - index_granularity = 14;" + index_granularity = 14, ratio_of_defaults_for_sparse_serialization = 1;" $CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES mt_compact" diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index 15f8821d80d..0d4acba338a 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -10,7 +10,7 @@ CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_qu $CLICKHOUSE_CLIENT -q "drop table if exists test_index" $CLICKHOUSE_CLIENT -q "drop table if exists idx" -$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into test_index select number, number > 3 ? 3 : number, number = 1 ? 1 : 0, number from numbers(20)" $CLICKHOUSE_CLIENT -q " @@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -q " explain actions = 1 select x from test_index where x > 15 order by x desc; " | grep -A 100 "ReadFromMergeTree" -$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into idx select number, number, number from numbers(10)" $CLICKHOUSE_CLIENT -q " diff --git a/tests/queries/0_stateless/02263_lazy_mark_load.sh b/tests/queries/0_stateless/02263_lazy_mark_load.sh index bf37556bfa6..35a1b4a44dd 100755 --- a/tests/queries/0_stateless/02263_lazy_mark_load.sh +++ b/tests/queries/0_stateless/02263_lazy_mark_load.sh @@ -24,7 +24,7 @@ CREATE TABLE lazy_mark_test n9 UInt64 ) ENGINE = MergeTree -ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0; +ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test" diff --git a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh index 9d1483f5bf7..76c562c9744 100755 --- a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh +++ b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") -${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x" +${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x SETTINGS ratio_of_defaults_for_sparse_serialization = 1" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 0 from numbers(1e6)" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 1 from numbers(1e6)" @@ -17,13 +17,11 @@ ${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 -q "optimize table tt final" "-- # Here SelectRows and SelectBytes should be zero, MergedRows is 2m and MergedUncompressedBytes is 16m ${CLICKHOUSE_CLIENT} -q "system flush logs" -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" ${CLICKHOUSE_CLIENT} --mutations_sync 1 -q "alter table tt update y = y + 1 where 1" "--query_id=$query_id" ${CLICKHOUSE_CLIENT} -q "system flush logs" # Here for mutation all values are 0, cause mutation is executed async. # It's pretty hard to write a test with total counter. -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" - - +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 5b603133f6c..e150d70b896 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -12,9 +12,10 @@ $CLICKHOUSE_CLIENT -nm -q " create table data_fsync_pe (key Int) engine=MergeTree() order by key settings - min_rows_for_wide_part=2, - fsync_after_insert=1, - fsync_part_directory=1; + min_rows_for_wide_part = 2, + fsync_after_insert = 1, + fsync_part_directory = 1, + ratio_of_defaults_for_sparse_serialization = 1; " ret=1 diff --git a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql index 842e22ba87d..2fe0943745d 100644 --- a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql +++ b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql @@ -1,12 +1,12 @@ -- Tags: no-upgrade-check, no-random-merge-tree-settings drop table if exists test_02381; -create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks=false, compress_primary_key=false; +create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381 select number, number * 10 from system.numbers limit 1000000; drop table if exists test_02381_compress; create table test_02381_compress(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) - SETTINGS compress_marks=true, compress_primary_key=true, marks_compression_codec='ZSTD(3)', primary_key_compression_codec='ZSTD(3)', marks_compress_block_size=65536, primary_key_compress_block_size=65536; + SETTINGS compress_marks = true, compress_primary_key = true, marks_compression_codec = 'ZSTD(3)', primary_key_compression_codec = 'ZSTD(3)', marks_compress_block_size = 65536, primary_key_compress_block_size = 65536, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381_compress select number, number * 10 from system.numbers limit 1000000; select * from test_02381_compress where a = 1000 limit 1; From e8f7a84ca6c4e00f6f9ddbf282b109f491244c4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:37:10 +0200 Subject: [PATCH 027/871] Update a few tests --- tests/queries/0_stateless/01375_compact_parts_codecs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql index 1dd39e67876..1c89eb09d0b 100644 --- a/tests/queries/0_stateless/01375_compact_parts_codecs.sql +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS codecs; CREATE TABLE codecs (id UInt32, val UInt32, s String) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -21,7 +21,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -38,7 +38,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts From 7c03801bf7da6803e47f57ab78478c33a9c9a764 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:54:57 +0200 Subject: [PATCH 028/871] 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 ea3e4219e35..ac29ef3f361 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 a25de5fb4186fbe103f916b07aa8bd89975048b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:55:44 +0200 Subject: [PATCH 029/871] 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 40f2c0ee400..88c910e0313 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 63b559df17a07e42768c4425538426e245d829fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 06:49:41 +0200 Subject: [PATCH 030/871] 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 40f2c0ee400..88c910e0313 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 383fc06761f81bee735ec22692a2d506ca78c01e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 9 May 2023 16:10:53 +0200 Subject: [PATCH 031/871] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 37 +++--- .../MaterializedPostgreSQLSettings.h | 3 + .../PostgreSQLReplicationHandler.cpp | 21 +-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 7 +- tests/integration/helpers/postgres_utility.py | 124 +++++++++++------- .../test.py | 75 ++++++++++- 6 files changed, 183 insertions(+), 84 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index d048c94ac75..ea7009fc082 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -562,34 +562,27 @@ void MaterializedPostgreSQLConsumer::syncTables() Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); - try + if (result_rows.rows()) { - if (result_rows.rows()) - { - auto storage = storage_data.storage; + auto storage = storage_data.storage; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); - auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = storage_data.buffer.columns_ast; - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto io = interpreter.execute(); - auto input = std::make_shared( - result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto io = interpreter.execute(); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); - io.pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index e8d42ef3668..d3d2faba497 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -21,6 +21,9 @@ namespace DB M(Bool, materialized_postgresql_tables_list_with_schema, false, \ "Consider by default that if there is a dot in tables list 'name.name', " \ "then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \ + M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \ + M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \ + M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 998db4ea79e..f57a6a26a62 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -22,8 +22,6 @@ namespace DB { -static const auto RESCHEDULE_MS = 1000; -static const auto BACKOFF_TRESHOLD_MS = 10000; static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min namespace ErrorCodes @@ -80,7 +78,10 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) - , milliseconds_to_wait(RESCHEDULE_MS) + , reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms) + , reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms) + , reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor) + , milliseconds_to_wait(reschedule_backoff_min_ms) { if (!schema_list.empty() && !tables_list.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and tables list at the same time"); @@ -166,7 +167,7 @@ void PostgreSQLReplicationHandler::checkConnectionAndStart() throw; LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); - startup_task->scheduleAfter(RESCHEDULE_MS); + startup_task->scheduleAfter(milliseconds_to_wait); } catch (...) { @@ -435,18 +436,18 @@ void PostgreSQLReplicationHandler::consumerFunc() if (schedule_now) { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = reschedule_backoff_min_ms; consumer_task->schedule(); LOG_DEBUG(log, "Scheduling replication thread: now"); } else { - consumer_task->scheduleAfter(milliseconds_to_wait); - if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS) - milliseconds_to_wait *= 2; + if (milliseconds_to_wait < reschedule_backoff_max_ms) + milliseconds_to_wait = std::min(milliseconds_to_wait * reschedule_backoff_factor, reschedule_backoff_max_ms); LOG_DEBUG(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); + consumer_task->scheduleAfter(milliseconds_to_wait); } } @@ -892,7 +893,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, @@ -922,7 +923,7 @@ void PostgreSQLReplicationHandler::removeTableFromReplication(const String & pos catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 10a196cf31b..4c16ff95692 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -140,13 +140,16 @@ private: BackgroundSchedulePool::TaskHolder consumer_task; BackgroundSchedulePool::TaskHolder cleanup_task; + const UInt64 reschedule_backoff_min_ms; + const UInt64 reschedule_backoff_max_ms; + const UInt64 reschedule_backoff_factor; + UInt64 milliseconds_to_wait; + std::atomic stop_synchronization = false; /// MaterializedPostgreSQL tables. Used for managing all operations with its internal nested tables. MaterializedStorages materialized_storages; - UInt64 milliseconds_to_wait; - bool replication_handler_initialized = false; }; diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index dfae37af434..1a00faf0f9d 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -76,16 +76,24 @@ def drop_postgres_schema(cursor, schema_name): def create_postgres_table( - cursor, table_name, replica_identity_full=False, template=postgres_table_template + cursor, + table_name, + database_name="", + replica_identity_full=False, + template=postgres_table_template, ): - drop_postgres_table(cursor, table_name) - cursor.execute(template.format(table_name)) + if database_name == "": + name = table_name + else: + name = f"{database_name}.{table_name}" + drop_postgres_table(cursor, name) + cursor.execute(template.format(name)) if replica_identity_full: - cursor.execute(f"ALTER TABLE {table_name} REPLICA IDENTITY FULL;") + cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") -def drop_postgres_table(cursor, table_name): - cursor.execute(f"""DROP TABLE IF EXISTS "{table_name}" """) +def drop_postgres_table(cursor, name): + cursor.execute(f"""DROP TABLE IF EXISTS "{name}" """) def create_postgres_table_with_schema(cursor, schema_name, table_name): @@ -103,13 +111,16 @@ class PostgresManager: self.created_materialized_postgres_db_list = set() self.created_ch_postgres_db_list = set() - def init(self, instance, ip, port): + def init(self, instance, ip, port, default_database="postgres_database"): self.instance = instance self.ip = ip self.port = port - self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.default_database = default_database self.prepare() + def get_default_database(self): + return self.default_database + def restart(self): try: self.clear() @@ -119,10 +130,17 @@ class PostgresManager: raise ex def prepare(self): - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() - self.create_postgres_db(cursor, "postgres_database") - self.create_clickhouse_postgres_db(ip=self.ip, port=self.port) + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() + if self.default_database != "": + self.create_postgres_db(self.default_database) + self.conn = get_postgres_conn( + ip=self.ip, + port=self.port, + database=True, + database_name=self.default_database, + ) + self.cursor = self.conn.cursor() def clear(self): if self.conn.closed == 0: @@ -132,63 +150,76 @@ class PostgresManager: for db in self.created_ch_postgres_db_list.copy(): self.drop_clickhouse_postgres_db(db) if len(self.created_postgres_db_list) > 0: - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() for db in self.created_postgres_db_list.copy(): - self.drop_postgres_db(cursor, db) + self.drop_postgres_db(db) - def get_db_cursor(self): - self.conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) + def get_db_cursor(self, database_name=""): + if database_name == "": + database_name = self.default_database + self.conn = get_postgres_conn( + ip=self.ip, port=self.port, database=True, database_name=database_name + ) return self.conn.cursor() - def create_postgres_db(self, cursor, name="postgres_database"): - self.drop_postgres_db(cursor, name) - self.created_postgres_db_list.add(name) - cursor.execute(f"CREATE DATABASE {name}") + def database_or_default(self, database_name): + if database_name == "" and self.default_database == "": + raise Exception("Database name is empty") + if database_name == "": + database_name = self.default_database + return database_name - def drop_postgres_db(self, cursor, name="postgres_database"): - cursor.execute(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_postgres_db_list: - self.created_postgres_db_list.remove(name) + def create_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.drop_postgres_db(database_name) + self.created_postgres_db_list.add(database_name) + self.cursor.execute(f"CREATE DATABASE {database_name}") + + def drop_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_postgres_db_list: + self.created_postgres_db_list.remove(database_name) def create_clickhouse_postgres_db( self, - ip, - port, - name="postgres_database", - database_name="postgres_database", + database_name="", schema_name="", ): - self.drop_clickhouse_postgres_db(name) - self.created_ch_postgres_db_list.add(name) + database_name = self.database_or_default(database_name) + self.drop_clickhouse_postgres_db(database_name) + self.created_ch_postgres_db_list.add(database_name) if len(schema_name) == 0: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) - def drop_clickhouse_postgres_db(self, name="postgres_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_ch_postgres_db_list: - self.created_ch_postgres_db_list.remove(name) + def drop_clickhouse_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.instance.query(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_ch_postgres_db_list: + self.created_ch_postgres_db_list.remove(database_name) def create_materialized_db( self, ip, port, materialized_database="test_database", - postgres_database="postgres_database", + postgres_database="", settings=[], table_overrides="", ): + postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") @@ -207,17 +238,12 @@ class PostgresManager: self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) - assert materialized_database not in self.instance.query("SHOW DATABASES") - def create_and_fill_postgres_table(self, table_name): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_table_from_cursor(cursor, table_name) - - def create_and_fill_postgres_table_from_cursor(self, cursor, table_name): - create_postgres_table(cursor, table_name) + def create_and_fill_postgres_table(self, table_name, database_name=""): + create_postgres_table(self.cursor, table_name, database_name) + database_name = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers(50)" + f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) def create_and_fill_postgres_tables(self, tables_num, numbers=50): diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 90d19e9532c..3b5194e8806 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -46,14 +46,34 @@ instance = cluster.add_instance( stay_alive=True, ) +instance2 = cluster.add_instance( + "instance2", + main_configs=["configs/log_conf.xml", "configs/merge_tree_too_many_parts.xml"], + user_configs=["configs/users.xml"], + with_postgres=True, + stay_alive=True, +) + + pg_manager = PostgresManager() +pg_manager2 = PostgresManager() @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="test_database", + ) + pg_manager.create_clickhouse_postgres_db() + pg_manager2.init( + instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + ) + pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -649,6 +669,59 @@ def test_materialized_view(started_cluster): pg_manager.drop_materialized_db() +def test_too_many_parts(started_cluster): + table = "test_table" + pg_manager2.create_and_fill_postgres_table(table) + pg_manager2.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = 'test_table', materialized_postgresql_backoff_min_ms = 100, materialized_postgresql_backoff_max_ms = 100" + ], + ) + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + assert ( + "50" == instance2.query("SELECT count() FROM test_database.test_table").strip() + ) + + instance2.query("SYSTEM STOP MERGES") + num = 50 + for i in range(10): + instance2.query( + f""" + INSERT INTO {pg_manager2.get_default_database()}.test_table SELECT {num}, {num}; + """ + ) + num = num + 1 + for i in range(30): + if num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) or instance2.contains_in_log("DB::Exception: Too many parts"): + break + time.sleep(1) + print(f"wait sync try {i}") + if instance2.contains_in_log("DB::Exception: Too many parts"): + num = num - 1 + break + assert num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) + + assert instance2.contains_in_log("DB::Exception: Too many parts") + print(num) + assert num == int(instance2.query("SELECT count() FROM test_database.test_table")) + + instance2.query("SYSTEM START MERGES") + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + + # assert "200" == instance.query("SELECT count FROM test_database.test_table").strip() + pg_manager2.drop_materialized_db() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From f0bfdb6b029748a486e5f683171f135d6a5dd957 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 22 Feb 2023 16:24:02 +0000 Subject: [PATCH 032/871] Refactor Query Tree visitor --- src/Analyzer/InDepthQueryTreeVisitor.h | 65 +++++++++++++++++++++++ src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++++++++---- 2 files changed, 88 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index 1cc48fb1e53..ee321842ffa 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -235,4 +235,69 @@ public: template using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; +template +class QueryTreeVisitor +{ +public: + explicit QueryTreeVisitor(ContextPtr context_) + : current_context(std::move(context_)) + {} + + bool needApply(QueryTreeNodePtr & node) + { + return getImpl().needApply(node); + } + + void visit(QueryTreeNodePtr & node) + { + auto current_scope_context_ptr = current_context; + SCOPE_EXIT( + current_context = std::move(current_scope_context_ptr); + ); + + if (auto * query_node = node->template as()) + current_context = query_node->getContext(); + else if (auto * union_node = node->template as()) + current_context = union_node->getContext(); + + if (!TOP_TO_BOTTOM) + visitChildren(node); + + if (needApply(node)) + getImpl().apply(node); + + if (TOP_TO_BOTTOM) + visitChildren(node); + } + + const ContextPtr & getContext() const + { + return current_context; + } + + const Settings & getSettings() const + { + return current_context->getSettingsRef(); + } +private: + + Impl & getImpl() + { + return *static_cast(this); + } + + void visitChildren(QueryTreeNodePtr & node) + { + for (auto & child : node->getChildren()) + { + if (child) + visit(child); + } + } + + static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; + + ContextPtr current_context; +}; + } diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 945295f5cbc..38f7d07d052 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,16 +16,17 @@ namespace DB namespace { -class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext +class CountDistinctVisitor : public QueryTreeVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + using QueryTreeVisitor::QueryTreeVisitor; - void visitImpl(QueryTreeNodePtr & node) + static constexpr bool TOP_TO_BOTTOM = true; + + bool needApply(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return; + return false; auto * query_node = node->as(); @@ -33,32 +34,43 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return; + return false; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return; + return false; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return; + return false; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return; + return false; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return; + return false; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return; + return false; + + return true; + } + + void apply(QueryTreeNodePtr & node) + { + auto * query_node = node->as(); + auto & projection_nodes = query_node->getProjection().getNodes(); + auto * function_node = projection_nodes[0]->as(); + + auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); From 5c34ee3019199a7e1d24730684c9c84e6c8e0615 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 9 May 2023 15:14:49 +0000 Subject: [PATCH 033/871] Skip unresolved table function arguments --- src/Analyzer/InDepthQueryTreeVisitor.h | 15 ++++++++++++++- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Analyzer/TableFunctionNode.cpp | 3 ++- src/Analyzer/TableFunctionNode.h | 8 +++++++- 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index ee321842ffa..be3a760d4e6 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -248,6 +249,16 @@ public: return getImpl().needApply(node); } + bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) + { + if (auto * table_function_node = parent->as()) + { + const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); + return std::find(unresolved_indexes.begin(), unresolved_indexes.end(), subtree_index) != unresolved_indexes.end(); + } + return false; + } + void visit(QueryTreeNodePtr & node) { auto current_scope_context_ptr = current_context; @@ -288,10 +299,12 @@ private: void visitChildren(QueryTreeNodePtr & node) { + size_t index = 0; for (auto & child : node->getChildren()) { - if (child) + if (child && !shouldSkipSubtree(node, index)) visit(child); + ++index; } } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7ab0261850b..aaea81dcada 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6356,7 +6356,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, table_function_ptr->parseArguments(table_function_ast, scope_context); auto table_function_storage = table_function_ptr->execute(table_function_ast, scope_context, table_function_ptr->getName()); - table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context); + table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context, std::move(skip_analysis_arguments_indexes)); } /// Resolve array join node in scope diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index c130503d660..30644ad4ec4 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -27,12 +27,13 @@ TableFunctionNode::TableFunctionNode(String table_function_name_) children[arguments_child_index] = std::make_shared(); } -void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context) +void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_) { table_function = std::move(table_function_value); storage = std::move(storage_value); storage_id = storage->getStorageID(); storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); + unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_); } const StorageID & TableFunctionNode::getStorageID() const diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 7786ba62205..69237ac8416 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -98,7 +98,7 @@ public: } /// Resolve table function with table function, storage and context - void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context); + void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_); /// Get storage id, throws exception if function node is not resolved const StorageID & getStorageID() const; @@ -106,6 +106,11 @@ public: /// Get storage snapshot, throws exception if function node is not resolved const StorageSnapshotPtr & getStorageSnapshot() const; + const std::vector & getUnresolvedArgumentIndexes() const + { + return unresolved_arguments_indexes; + } + /// Return true if table function node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -164,6 +169,7 @@ private: StoragePtr storage; StorageID storage_id; StorageSnapshotPtr storage_snapshot; + std::vector unresolved_arguments_indexes; std::optional table_expression_modifiers; SettingsChanges settings_changes; From f9eb6ca6fd767666b7b2c8e88e92e8d02639ef99 Mon Sep 17 00:00:00 2001 From: xiebin Date: Wed, 10 May 2023 23:43:40 +0800 Subject: [PATCH 034/871] if the data type of numeric key is not native uint, convert to complex. --- .../getDictionaryConfigurationFromAST.cpp | 16 ++++++++++------ ...y_layout_to_complex_by_complex_keys.reference | 1 + ...tionary_layout_to_complex_by_complex_keys.sql | 5 +++++ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 59f1a712d00..c179ce84ff9 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -609,17 +610,20 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type); - if (pk_attrs.size() > 1 && !complex - && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) - { - complex = true; - } - auto all_attr_names_and_types = buildDictionaryAttributesConfiguration( xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); checkPrimaryKey(all_attr_names_and_types, pk_attrs); + /// If the pk size is 1 and pk's DataType is not native uint(UInt8~UInt64), we should convert to complex, + /// because the data type of Numeric key(simple layout) is UInt64. + if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !WhichDataType(DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type)).isNativeUInt())) + && !complex + && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) + { + complex = true; + } + buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list); buildLayoutConfiguration(xml_document, current_dictionary, query.dictionary->dict_settings, dictionary_layout); diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference index 7f38556e7d1..cc9381622ec 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -2,3 +2,4 @@ dict_flat_simple Flat dict_hashed_simple Hashed dict_hashed_complex ComplexKeyHashed dict_hashed_simple_auto_convert ComplexKeyHashed +dict_hashed_simple_int_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 0fb06e5acc2..77d933afa9f 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -26,4 +26,9 @@ SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; DROP DICTIONARY dict_hashed_simple_auto_convert; +CREATE DICTIONARY dict_hashed_simple_int_auto_convert (v0 Int16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_int_auto_convert; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_int_auto_convert'; +DROP DICTIONARY dict_hashed_simple_int_auto_convert; + DROP TABLE dict_data; From 6b0bd698d36014a5eac052857bac2185a1f45f41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:17:53 +0200 Subject: [PATCH 035/871] Fix mistake --- .../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 88c910e0313..40f2c0ee400 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) 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) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 65d28a959ff5b21199c2b20d8dcb7c7b399f314d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:26:29 +0200 Subject: [PATCH 036/871] Update integration tests (1/2) --- .../configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_hdfs/configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_s3_failover/configs/config.xml | 4 ++++ .../test_s3_zero_copy_replication/configs/config.d/s3.xml | 1 + 4 files changed, 7 insertions(+) diff --git a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml index cb87abcc693..d69fe96a3e2 100644 --- a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml @@ -45,5 +45,6 @@ true + 1.0 diff --git a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml index 890c396ed95..7d59081486b 100644 --- a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml @@ -28,5 +28,6 @@ 0 + 1.0 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml index feb537ebbce..743d75d9a21 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.xml @@ -15,4 +15,8 @@ 500 ./clickhouse/ users.xml + + + 1.0 + diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index f7d9efc2cae..55c35999703 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -70,6 +70,7 @@ 1024 1 true + 1.0 From c9e752fdc5c4cc401df240f7cd5f77586d9b542d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 11 May 2023 18:09:46 +0200 Subject: [PATCH 037/871] Fix --- tests/integration/helpers/postgres_utility.py | 51 ++++---- .../test.py | 113 +++++++----------- .../test.py | 64 +++++----- 3 files changed, 99 insertions(+), 129 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 1a00faf0f9d..3c8a23b15a2 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -87,7 +87,9 @@ def create_postgres_table( else: name = f"{database_name}.{table_name}" drop_postgres_table(cursor, name) - cursor.execute(template.format(name)) + query = template.format(name) + cursor.execute(query) + print(f"Query: {query}") if replica_identity_full: cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") @@ -129,6 +131,9 @@ class PostgresManager: self.prepare() raise ex + def execute(self, query): + self.cursor.execute(query) + def prepare(self): self.conn = get_postgres_conn(ip=self.ip, port=self.port) self.cursor = self.conn.cursor() @@ -141,6 +146,7 @@ class PostgresManager: database_name=self.default_database, ) self.cursor = self.conn.cursor() + self.create_clickhouse_postgres_db() def clear(self): if self.conn.closed == 0: @@ -164,11 +170,11 @@ class PostgresManager: return self.conn.cursor() def database_or_default(self, database_name): - if database_name == "" and self.default_database == "": - raise Exception("Database name is empty") - if database_name == "": - database_name = self.default_database - return database_name + if database_name != "": + return database_name + if self.default_database != "": + return self.default_database + raise Exception("Database name is empty") def create_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) @@ -186,8 +192,11 @@ class PostgresManager: self, database_name="", schema_name="", + postgres_database="", ): database_name = self.database_or_default(database_name) + if postgres_database == "": + postgres_database = database_name self.drop_clickhouse_postgres_db(database_name) self.created_ch_postgres_db_list.add(database_name) @@ -195,13 +204,13 @@ class PostgresManager: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) def drop_clickhouse_postgres_db(self, database_name=""): @@ -239,6 +248,16 @@ class PostgresManager: if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) + def create_postgres_schema(self, name): + create_postgres_schema(self.cursor, name) + + def create_postgres_table( + self, table_name, database_name="", template=postgres_table_template + ): + create_postgres_table( + self.cursor, table_name, database_name=database_name, template=template + ) + def create_and_fill_postgres_table(self, table_name, database_name=""): create_postgres_table(self.cursor, table_name, database_name) database_name = self.database_or_default(database_name) @@ -246,22 +265,14 @@ class PostgresManager: f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) - def create_and_fill_postgres_tables(self, tables_num, numbers=50): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_tables_from_cursor( - cursor, tables_num, numbers=numbers - ) - - def create_and_fill_postgres_tables_from_cursor( - self, cursor, tables_num, numbers=50 - ): + def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name=""): for i in range(tables_num): table_name = f"postgresql_replica_{i}" - create_postgres_table(cursor, table_name) + create_postgres_table(self.cursor, table_name, database_name) if numbers > 0: + db = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers({numbers})" + f"INSERT INTO {db}.{table_name} SELECT number, number from numbers({numbers})" ) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 377b1c89efc..1eb2efc73a5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -46,7 +46,12 @@ pg_manager = PostgresManager() def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + ) yield cluster finally: @@ -74,16 +79,10 @@ def test_load_and_sync_all_database_tables(started_cluster): def test_replicating_dml(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table(cursor, "postgresql_replica_{}".format(i)) + pg_manager.create_postgres_table(f"postgresql_replica_{i}") instance.query( "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format( i, i @@ -96,39 +95,29 @@ def test_replicating_dml(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format( - i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 50 + number, {i} from numbers(1000)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;".format( - i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} WHERE key < 50;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} * {} WHERE key >= 50;".format( - i, i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} * {i} WHERE key >= 50;" ) + check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;".format( - i, i - ) + pg_manager.execute( + f"DELETE FROM postgresql_replica_{i} WHERE (value*value + {i}) % 2 = 0;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = value - (value % 7) WHERE key > 128 AND key < 512;".format( - i - ) - ) - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE key % 7 = 1;".format(i, i) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = value - (value % 7) WHERE key > 128 AND key < 512;" ) + pg_manager.execute(f"DELETE FROM postgresql_replica_{i} WHERE key % 7 = 1;") + check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -288,13 +277,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): def test_changing_replica_identity_value(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") + pg_manager.create_postgres_table("postgresql_replica") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)" ) @@ -307,7 +290,7 @@ def test_changing_replica_identity_value(started_cluster): "INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)" ) check_tables_are_synchronized(instance, "postgresql_replica") - cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") check_tables_are_synchronized(instance, "postgresql_replica") @@ -331,18 +314,13 @@ def test_clickhouse_restart(started_cluster): def test_replica_identity_index(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, + pg_manager.create_postgres_table( + "postgresql_replica", template=postgres_table_template_3 ) - cursor = conn.cursor() - - create_postgres_table( - cursor, "postgresql_replica", template=postgres_table_template_3 + pg_manager.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") + pg_manager.execute( + "ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx" ) - cursor.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") - cursor.execute("ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(50, 10)" ) @@ -355,35 +333,29 @@ def test_replica_identity_index(started_cluster): ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") - cursor.execute("UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 ") + pg_manager.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") + pg_manager.execute( + "UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 " + ) + pg_manager.execute( + "UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 " + ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("DELETE FROM postgresql_replica WHERE key2<75;") + pg_manager.execute("DELETE FROM postgresql_replica WHERE key2<75;") check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") def test_table_schema_changes(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table( - cursor, - "postgresql_replica_{}".format(i), - template=postgres_table_template_2, + pg_manager.create_postgres_table( + f"postgresql_replica_{i}", template=postgres_table_template_2 ) instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT number, {i}, {i}, {i} from numbers(25)" ) pg_manager.create_materialized_db( @@ -393,9 +365,7 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 25 + number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 25 + number, {i}, {i}, {i} from numbers(25)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -444,10 +414,7 @@ def test_many_concurrent_queries(started_cluster): port=started_cluster.postgres_port, database=True, ) - cursor = conn.cursor() - pg_manager.create_and_fill_postgres_tables_from_cursor( - cursor, NUM_TABLES, numbers=10000 - ) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000) def attack(thread_id): print("thread {}".format(thread_id)) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..2b17024f417 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -67,13 +67,11 @@ def started_cluster(): instance, cluster.postgres_ip, cluster.postgres_port, - default_database="test_database", + default_database="postgres_database", ) - pg_manager.create_clickhouse_postgres_db() pg_manager2.init( - instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) - pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -88,11 +86,10 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") + pg_manager.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( @@ -158,7 +155,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_6" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -169,7 +166,7 @@ def test_add_new_table_to_replication(started_cluster): instance.restart_clickhouse() table_name = "postgresql_replica_7" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -271,8 +268,7 @@ def test_remove_table_from_replication(started_cluster): == ")\\nSETTINGS materialized_postgresql_tables_list = \\'postgresql_replica_0,postgresql_replica_2,postgresql_replica_3,postgresql_replica_4\\'\n" ) - cursor = pg_manager.get_db_cursor() - cursor.execute(f"drop table if exists postgresql_replica_0;") + pg_manager.execute(f"drop table if exists postgresql_replica_0;") # Removing from replication table which does not exist in PostgreSQL must be ok. instance.query("DETACH TABLE test_database.postgresql_replica_0 PERMANENTLY") @@ -282,10 +278,11 @@ def test_remove_table_from_replication(started_cluster): def test_predefined_connection_configuration(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" ) @@ -332,10 +329,9 @@ def test_database_with_single_non_default_schema(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -367,7 +363,7 @@ def test_database_with_single_non_default_schema(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -434,10 +430,9 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -472,7 +467,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -550,10 +545,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, - schema_name=schema_name, + database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" @@ -586,7 +578,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): altered_schema = random.randint(0, schemas_num - 1) altered_table = random.randint(0, NUM_TABLES - 1) clickhouse_postgres_db = f"clickhouse_postgres_db{altered_schema}" - cursor.execute( + pg_manager.execute( f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer" ) @@ -619,10 +611,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): def test_table_override(started_cluster): - cursor = pg_manager.get_db_cursor() table_name = "table_override" materialized_database = "test_database" - create_postgres_table(cursor, table_name, template=postgres_table_template_5) + pg_manager.create_postgres_table(table_name, template=postgres_table_template_5) instance.query( f"create table {table_name}(key Int32, value UUID) engine = PostgreSQL (postgres1, table={table_name})" ) @@ -649,10 +640,11 @@ def test_table_override(started_cluster): def test_materialized_view(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query("DROP DATABASE IF EXISTS test_database") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" @@ -663,7 +655,7 @@ def test_materialized_view(started_cluster): "CREATE MATERIALIZED VIEW mv ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT * FROM test_database.test_table" ) assert "1\t2" == instance.query("SELECT * FROM mv").strip() - cursor.execute(f"INSERT INTO test_table SELECT 3, 4") + pg_manager.execute(f"INSERT INTO test_table SELECT 3, 4") check_tables_are_synchronized(instance, "test_table") assert "1\t2\n3\t4" == instance.query("SELECT * FROM mv ORDER BY 1, 2").strip() pg_manager.drop_materialized_db() From e2112576f04d3aeda1bc5384b5c49da4dc2a8e0c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 16 May 2023 12:21:32 +0200 Subject: [PATCH 038/871] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2b17024f417..acec01e732b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -545,7 +545,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", + database_name=clickhouse_postgres_db, + schema_name=schema_name, + postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" From 1eb939766bc78a59dd11b3534f4fd7b693d75e21 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 16 May 2023 17:48:49 +0000 Subject: [PATCH 039/871] 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 4592e92151e..e176c7afd76 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 455e486c798..b22b0c73907 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -38,6 +38,7 @@ public: 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 00000000000..573541ac970 --- /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 00000000000..4685e49b96d --- /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 849cddd8b281be1ecebf104f5b8f6670c1c6e916 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 May 2023 12:33:26 +0200 Subject: [PATCH 040/871] Add forgotten file --- .../configs/merge_tree_too_many_parts.xml | 5 +++++ .../test_postgresql_replica_database_engine_2/test.py | 7 ++----- 2 files changed, 7 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml diff --git a/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml new file mode 100644 index 00000000000..4bc63453f55 --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml @@ -0,0 +1,5 @@ + + + 5 + + diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..07f356250ea 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -88,11 +88,8 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( From 05a90a2e971ae7538ed72e1a3db02523c91b67d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 17 May 2023 12:19:00 +0000 Subject: [PATCH 041/871] 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 ec245d8b9e0..eb8b912f03b 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 09cc62dac00..85cdc278892 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 042/871] 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 5bbc8eae0de..76e4ea1ebd1 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 0ca3e46db88..2f0046bc39c 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,6 +92,9 @@ public: 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 @@ private: 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 de7011b67bf..896bd5949d9 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 4780dfed4b2..2634439ee14 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 043/871] 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 76e4ea1ebd1..3f9535205d8 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 2f0046bc39c..bc2f923f705 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,8 +92,8 @@ public: 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 @@ private: 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 896bd5949d9..a4d2cb3d305 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 2634439ee14..4b1cd4e036e 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 044/871] 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 3f9535205d8..fdfc6343876 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 bc2f923f705..479a0053efa 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,7 @@ public: 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 53b6163ba87..00cbfbba659 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 beb3690c7ecf95f496f4cea2be7d3b19ae24046c Mon Sep 17 00:00:00 2001 From: xiebin Date: Tue, 30 May 2023 16:09:01 +0800 Subject: [PATCH 045/871] if dictionary id is number, do not convert layout to complex --- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 7 ++++--- .../queries/0_stateless/02391_hashed_dictionary_shards.sql | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index c179ce84ff9..5f8e0bc27b4 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -615,9 +615,10 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte checkPrimaryKey(all_attr_names_and_types, pk_attrs); - /// If the pk size is 1 and pk's DataType is not native uint(UInt8~UInt64), we should convert to complex, - /// because the data type of Numeric key(simple layout) is UInt64. - if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !WhichDataType(DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type)).isNativeUInt())) + /// If the pk size is 1 and pk's DataType is not number, we should convert to complex. + /// NOTE: the data type of Numeric key(simple layout) is UInt64, so if the type is not under UInt64, type casting will lead to precision loss. + DataTypePtr first_key_type = DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type); + if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !isNumber(first_key_type))) && !complex && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) { diff --git a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql index ac43c12afc0..018f6b2cf4f 100644 --- a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql +++ b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql @@ -89,7 +89,7 @@ SOURCE(CLICKHOUSE(TABLE test_table_string)) LAYOUT(SPARSE_HASHED(SHARDS 10)) LIFETIME(0); -SYSTEM RELOAD DICTIONARY test_dictionary_10_shards_string; -- { serverError CANNOT_PARSE_TEXT } +SYSTEM RELOAD DICTIONARY test_dictionary_10_shards_string; DROP DICTIONARY test_dictionary_10_shards_string; From b66881b95a3698c050ef0ff9bff2f570fa01fba8 Mon Sep 17 00:00:00 2001 From: xiebin Date: Tue, 30 May 2023 17:59:46 +0800 Subject: [PATCH 046/871] refine functional test --- ...ayout_to_complex_by_complex_keys.reference | 6 ++-- ...nary_layout_to_complex_by_complex_keys.sql | 35 ++++++++++--------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference index cc9381622ec..7616c59e4fd 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -1,5 +1,5 @@ dict_flat_simple Flat -dict_hashed_simple Hashed -dict_hashed_complex ComplexKeyHashed +dict_hashed_simple_Decimal128 Hashed +dict_hashed_simple_Float32 Hashed +dict_hashed_simple_String ComplexKeyHashed dict_hashed_simple_auto_convert ComplexKeyHashed -dict_hashed_simple_int_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 77d933afa9f..753b9f663b5 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -1,34 +1,35 @@ DROP DICTIONARY IF EXISTS dict_flat_simple; -DROP DICTIONARY IF EXISTS dict_hashed_simple; -DROP DICTIONARY IF EXISTS dict_hashed_complex; +DROP DICTIONARY IF EXISTS dict_hashed_simple_Decimal128; +DROP DICTIONARY IF EXISTS dict_hashed_simple_Float32; +DROP DICTIONARY IF EXISTS dict_hashed_simple_String; DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; DROP TABLE IF EXISTS dict_data; -CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory(); +CREATE TABLE dict_data (v0 UInt16, v1 Int16, v2 Float32, v3 Decimal128(10), v4 String) engine=Memory() AS SELECT number, number%65535, number*1.1, number*1.1, 'foo' FROM numbers(10);; CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); SYSTEM RELOAD DICTIONARY dict_flat_simple; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_flat_simple'; DROP DICTIONARY dict_flat_simple; -CREATE DICTIONARY dict_hashed_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_simple; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple'; -DROP DICTIONARY dict_hashed_simple; +CREATE DICTIONARY dict_hashed_simple_Decimal128 (v3 Decimal128(10), v1 UInt16, v2 Float32) PRIMARY KEY v3 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_Decimal128; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_Decimal128'; +DROP DICTIONARY dict_hashed_simple_Decimal128; -CREATE DICTIONARY dict_hashed_complex (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(complex_key_hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_complex; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_complex'; -DROP DICTIONARY dict_hashed_complex; +CREATE DICTIONARY dict_hashed_simple_Float32 (v2 Float32, v3 Decimal128(10), v4 String) PRIMARY KEY v2 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_Float32; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_Float32'; +DROP DICTIONARY dict_hashed_simple_Float32; -CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +CREATE DICTIONARY dict_hashed_simple_String (v4 String, v3 Decimal128(10), v2 Float32) PRIMARY KEY v4 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_String; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_String'; +DROP DICTIONARY dict_hashed_simple_String; + +CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 Int16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; DROP DICTIONARY dict_hashed_simple_auto_convert; -CREATE DICTIONARY dict_hashed_simple_int_auto_convert (v0 Int16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_simple_int_auto_convert; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_int_auto_convert'; -DROP DICTIONARY dict_hashed_simple_int_auto_convert; - DROP TABLE dict_data; From cd8eb44f0c54945f4777ed3e50e08b057ee41f43 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 31 May 2023 14:03:11 +0000 Subject: [PATCH 047/871] 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 fdfc6343876..b6db53018f4 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, ""); + if (text_node->nodeType() != Node::TEXT_NODE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have text node", node->nodeName()); - /* - 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); + 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 479a0053efa..c9b227863f0 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,13 @@ public: 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 048/871] 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 d095ab3a1be..12a2b59ff77 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 00000000000..cd48963c47a --- /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 00cbfbba659..80c3535ef4e 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 049/871] 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 b6db53018f4..055a497fb38 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 050/871] 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 055a497fb38..99bea019c3b 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 051/871] 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 cd48963c47a..542e173deb9 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); - 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 052/871] 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 99bea019c3b..df25a9a3825 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 053/871] 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 68fbfe0b22a..1ca2db44b13 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 054/871] 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 ded7a4643a9..0787ead76cf 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 1a361ef3060a4d271e6e27bb816df1d18cffaa02 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 5 Jun 2023 03:21:43 +0200 Subject: [PATCH 055/871] works for file --- src/Storages/StorageFile.cpp | 103 +++++++++++++++++++++++++++++++---- 1 file changed, 93 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 647f9511052..53da509d383 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -90,6 +90,57 @@ namespace ErrorCodes namespace { +/// Forward-declare to use in listFilesWithFoldedRegexpMatchingImpl() +void listFilesWithRegexpMatchingImpl( + const std::string & path_for_ls, + const std::string & for_match, + size_t & total_bytes_to_read, + std::vector & result, + bool recursive = false); + +/// When `{...}` has any `/`s, it must be processed in a different way +void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, + const std::string & suffix_with_globs, + const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, + const size_t max_depth, const size_t next_slash_after_glob_pos, + std::vector & result) +{ + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return; + + const fs::directory_iterator end; + for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + { + const std::string full_path = it->path().string(); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + total_bytes_to_read += it->file_size(); + result.push_back(it->path().string()); + } + else + { + listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + suffix_with_globs.substr(next_slash_after_glob_pos), + total_bytes_to_read, result); + } + } + else if (it->is_directory()) + { + listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, + glob, matcher, total_bytes_to_read, max_depth - 1, + next_slash_after_glob_pos, result); + } + + } +} + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ @@ -98,15 +149,40 @@ void listFilesWithRegexpMatchingImpl( const std::string & for_match, size_t & total_bytes_to_read, std::vector & result, - bool recursive = false) + bool recursive) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - const std::string current_glob = suffix_with_globs.substr(0, next_slash); + /// slashes_in_glob counter is a upper-bound estimate of recursion depth + /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + auto regexp = makeRegexpPatternFromGlobs(current_glob); re2::RE2 matcher(regexp); @@ -123,13 +199,22 @@ void listFilesWithRegexpMatchingImpl( if (!fs::exists(prefix_without_globs)) return; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; + + if (slashes_in_glob) + { + listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + current_glob, matcher, total_bytes_to_read, slashes_in_glob, + next_slash_after_glob_pos, result); + return; + } + const fs::directory_iterator end; for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; /// Condition is_directory means what kind of path is it in current iteration of ls if (!it->is_directory() && !looking_for_directory) @@ -145,14 +230,12 @@ void listFilesWithRegexpMatchingImpl( if (recursive) { listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash) : current_glob , + 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)) - { /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read, result); - } + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } } From d316add2f1f6ffa9cf6f2a1107a4d7d69960c72a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 6 Jun 2023 15:24:29 +0000 Subject: [PATCH 056/871] 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 00000000000..e69de29bb2d 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 00000000000..5c274128e39 --- /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 00000000000..a3cb1bb57f3 --- /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 057/871] 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 a64676cfa01..5496720e5dc 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 1a9f226041b..ca27cbdbf19 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 ece53808519ce6e42bbff2f2f3ac7d9ffccabe63 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 15:48:38 +0200 Subject: [PATCH 058/871] add simple stateless for file() --- .../02771_complex_globs_in_storage_file_path.reference | 4 ++++ .../02771_complex_globs_in_storage_file_path.sql | 8 ++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference new file mode 100644 index 00000000000..e1b420ecf37 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference @@ -0,0 +1,4 @@ +This is file data1 data1.csv +This is file data2 data2.csv +This is file data1 data1.csv +This is file data2 data2.csv diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql new file mode 100644 index 00000000000..c579c8d8698 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -0,0 +1,8 @@ +-- Tags: no-replicated-database, no-parallel + +INSERT INTO TABLE FUNCTION file('02771/dir1/subdir11/data1.csv', 'CSV', 's String') SELECT 'This is file data1' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's String') SELECT 'This is file data2' SETTINGS engine_file_truncate_on_insert=1; + +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); From bfdb18619c3b31044eba28f15cd9a0d7141b790e Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 19:25:33 +0200 Subject: [PATCH 059/871] small docs udpate --- docs/en/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 28c2dc9f1f3..09e3df7cc6a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -134,7 +134,7 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 94bc734a8fb..f0fea630ad2 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). From ff1b069cdb9301f5c13bed69d70c484210693dea Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Jun 2023 21:15:35 +0000 Subject: [PATCH 060/871] 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 33f40eef14e..98e78045b85 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 061/871] 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 4685e49b96d..5fafb773d16 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 062/871] 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 00000000000..49bf51b5bad --- /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 00000000000..07bf69d17c8 --- /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 00000000000..b0e519ff546 --- /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 00000000000..b452ce6374c --- /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 00000000000..c01f5050b00 --- /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 4a7761c16210c7e2eccc0b26e172ec8dc7e6c183 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 08:26:32 +0000 Subject: [PATCH 063/871] Add column is_obsolete for system.settings table and related system tables --- src/Interpreters/Context.cpp | 6 ++++-- src/Storages/System/StorageSystemMergeTreeSettings.cpp | 2 ++ src/Storages/System/StorageSystemServerSettings.cpp | 2 ++ src/Storages/System/StorageSystemSettings.cpp | 2 ++ tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- tests/queries/0_stateless/01945_system_warnings.sh | 4 ++-- 7 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 995e78d8f0b..a12117b7677 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -781,8 +781,10 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back("Some obsolete setting is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); + common_warnings.emplace_back( + "Obsolete setting `" + setting.getName() + + "` is changed. " + "Check 'select * from system.settings where changed' and read the changelog."); break; } } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 6de3fb800f4..0ddd4546208 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() {"max", std::make_shared(std::make_shared())}, {"readonly", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -52,6 +53,7 @@ void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); + res_columns[8]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index ad52c6896ac..290b575465c 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -15,6 +15,7 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -33,6 +34,7 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); + res_columns[6]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index c54f7eef25f..dcb54eac0a0 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList StorageSystemSettings::getNamesAndTypes() {"type", std::make_shared()}, {"default", std::make_shared()}, {"alias_for", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -51,6 +52,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr co res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getDefaultValueString()); + res_columns[10]->insert(setting.isObsolete()); }; const auto & settings_to_aliases = Settings::Traits::settingsToAliases(); diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 4e6dd3e1b0f..3d5b1ca99a5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Some obsolete setting is changed." +expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 296a03447db..d6ae567289c 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Some obsolete setting is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index bf11cee2911..112baab614e 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -14,8 +14,8 @@ else echo "Server was built in debug mode. It will work slowly." fi -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%obsolete setting%'" -${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%obsolete setting%'" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%Obsolete setting%'" +${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%Obsolete setting%'" # Avoid duplicated warnings ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" From 820673a5cf3f3f1c17b781496b3ab56f72f72c08 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 10:16:53 +0000 Subject: [PATCH 064/871] update test --- .../queries/0_stateless/01221_system_settings.reference | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 399b3778b66..e9c2f3fec32 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 -storage_policy default 0 Name of storage disk policy \N \N 0 String +send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 +storage_policy default 0 Name of storage disk policy \N \N 0 String 0 1 1 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 e864ba85018..38d00c15725 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -346,7 +346,8 @@ CREATE TABLE system.merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -918,7 +919,8 @@ CREATE TABLE system.replicated_merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -993,7 +995,8 @@ CREATE TABLE system.settings `readonly` UInt8, `type` String, `default` String, - `alias_for` String + `alias_for` String, + `is_obsolete` UInt8 ) ENGINE = SystemSettings COMMENT 'SYSTEM TABLE is built on the fly.' From 2c97a94892358f7e6b4a7aa4b25c4d46dd0f0fc0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 01:50:17 +0200 Subject: [PATCH 065/871] fix hdfs + style update --- src/Storages/HDFS/StorageHDFS.cpp | 115 ++++++++++++++++++++++++++++-- src/Storages/StorageFile.cpp | 20 +++--- 2 files changed, 120 insertions(+), 15 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19c0840149b..379ee395939 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,23 +63,124 @@ namespace ErrorCodes } namespace { + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times); + + /// When `{...}` has any `/`s, it must be processed in a different way + Strings LSWithFoldedRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + std::unordered_map * last_mod_times, + const String & processed_suffix, + const String & suffix_with_globs, + const String & current_glob, + re2::RE2 & matcher, + const size_t max_depth, + const size_t next_slash_after_glob_pos) + { + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return {}; + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + { + // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. + throw Exception( + ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); + } + + Strings result; + if (!ls.file_info && ls.length > 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); + + for (int i = 0; i < ls.length; ++i) + { + const String full_path = String(ls.file_info[i].mName); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + const bool is_directory = ls.file_info[i].mKind == 'D'; + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + result.push_back(String(ls.file_info[i].mName)); + if (last_mod_times) + (*last_mod_times)[result.back()] = ls.file_info[i].mLastMod; + } + else + { + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "" , fs, + suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + else if (is_directory) + { + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, + max_depth - 1, next_slash_after_glob_pos); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + return result; + } + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ - Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times) + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + + re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); if (!matcher.ok()) throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); + if (slashes_in_glob) + { + return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, last_mod_times, + "", suffix_with_globs, current_glob, matcher, + slashes_in_glob, next_slash_after_glob_pos); + } + HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); if (ls.file_info == nullptr && errno != ENOENT) // NOLINT @@ -96,7 +197,7 @@ namespace const String full_path = String(ls.file_info[i].mName); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; const bool is_directory = ls.file_info[i].mKind == 'D'; /// Condition with type of current file_info means what kind of path is it in current iteration of ls if (!is_directory && !looking_for_directory) @@ -112,7 +213,7 @@ namespace { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash), last_mod_times); + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 53da509d383..74303b16ee9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -99,10 +99,14 @@ void listFilesWithRegexpMatchingImpl( bool recursive = false); /// When `{...}` has any `/`s, it must be processed in a different way -void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, +void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, + const std::string & processed_suffix, const std::string & suffix_with_globs, - const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, - const size_t max_depth, const size_t next_slash_after_glob_pos, + const std::string & current_glob, + re2::RE2 & matcher, + size_t & total_bytes_to_read, + const size_t max_depth, + const size_t next_slash_after_glob_pos, std::vector & result) { /// We don't need to go all the way in every directory if max_depth is reached @@ -111,7 +115,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const return; const fs::directory_iterator end; - for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + for (fs::directory_iterator it(path_for_ls); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); @@ -133,9 +137,9 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, - glob, matcher, total_bytes_to_read, max_depth - 1, - next_slash_after_glob_pos, result); + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + suffix_with_globs, current_glob, matcher, + total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } } @@ -203,7 +207,7 @@ void listFilesWithRegexpMatchingImpl( if (slashes_in_glob) { - listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, current_glob, matcher, total_bytes_to_read, slashes_in_glob, next_slash_after_glob_pos, result); return; From aad7712b18fa358f60c59565d3914fafc2a3f759 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 02:09:05 +0200 Subject: [PATCH 066/871] add existing test --- tests/integration/test_storage_hdfs/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index edf5344e887..0b18c0180cc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -85,6 +85,15 @@ def test_read_write_storage_with_globs(started_cluster): assert "in readonly mode" in str(ex) +def test_storage_with_multidirectory_glob(started_cluster): + hdfs_api = started_cluster.hdfs_api + for i in ["1", "2"]: + hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") + assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + + assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + + def test_read_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api From 427c5cb1bafef9b52011f9d77e725b7a5ba85553 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sun, 11 Jun 2023 18:42:10 +0200 Subject: [PATCH 067/871] fix integration test --- tests/integration/test_storage_hdfs/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 0b18c0180cc..dde3dd257b1 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -91,7 +91,8 @@ def test_storage_with_multidirectory_glob(started_cluster): hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" - assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") def test_read_write_table(started_cluster): From eb9cdbcf7d5d1cdcc3b45936b4045dd8eda8f818 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:41:36 +0200 Subject: [PATCH 068/871] fix File test being flaky --- src/Storages/StorageFile.cpp | 15 +++++++++++---- .../02771_complex_globs_in_storage_file_path.sql | 4 +++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 74303b16ee9..c7a57b7d4c9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -109,8 +109,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /// We don't need to go all the way in every directory if max_depth is reached - /// as it is upper limit of depth by simply counting `/`s in curly braces + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. + */ if (!max_depth) return; @@ -121,6 +127,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); + std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) @@ -130,14 +137,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, } else { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "" , suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path), processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql index c579c8d8698..41d7d6fcc3f 100644 --- a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -5,4 +5,6 @@ INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's Strin SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); -SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); + +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; From 7d7bd5bb66fbf1e0d3a1e35a0fe74231d6e5e7ed Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:50:23 +0200 Subject: [PATCH 069/871] update comment describing workflow --- src/Storages/HDFS/StorageHDFS.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 379ee395939..bd50b66ede5 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,12 +63,20 @@ namespace ErrorCodes } namespace { + /// Forward-declared to use in LSWithFoldedRegexpMatching w/o circular dependency. Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times); - /// When `{...}` has any `/`s, it must be processed in a different way + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageFile.cpp has the same logic. + */ Strings LSWithFoldedRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, std::unordered_map * last_mod_times, From 3e6d393e17b0913d2664838a98a8f946bc8b7695 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 12:06:21 +0200 Subject: [PATCH 070/871] remove debug cerr --- src/Storages/StorageFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c7a57b7d4c9..93228cf4d39 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -127,7 +127,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); - std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) From cd1a3916a6ea755b24b475983f6f67447cebdd6a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 12 Jun 2023 13:28:17 +0200 Subject: [PATCH 071/871] cleanup HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index bd50b66ede5..7e836c028a0 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -129,7 +129,7 @@ namespace } else if (is_directory) { - Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path), fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, max_depth - 1, next_slash_after_glob_pos); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); From e9763caa0eb7078cd28e3765d0da1e0a9b4a204b Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 14:21:58 +0000 Subject: [PATCH 072/871] fix --- src/Interpreters/Context.cpp | 18 +++++++++++++----- .../01945_system_warnings.reference | 2 +- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a12117b7677..823c3d678df 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -777,17 +777,25 @@ Strings Context::getWarnings() const auto lock = getLock(); common_warnings = shared->warnings; } + String res = "Obsolete settings ["; + size_t obsolete_settings_count = 0; for (const auto & setting : settings) { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back( - "Obsolete setting `" + setting.getName() - + "` is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); - break; + res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + ++obsolete_settings_count; } } + + if (obsolete_settings_count) + { + res = res + "]" + (obsolete_settings_count == 1 ? " is" : " are") + + " changed. " + "Please check 'select * from system.settings where changed and is_obsolete' and read the changelog."; + common_warnings.emplace_back(res); + } + return common_warnings; } diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index d6ae567289c..3e7edacd275 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 From 18f4f1a5238c64f3b45e1d6781ef2c7104ab842d Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:11:19 +0000 Subject: [PATCH 073/871] udpate test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 3d5b1ca99a5..f0c97acb1f5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 3e7edacd275..0c05d5d7049 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From b76ba13250ad5b0abe728875be0e41667450cd5f Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:41:46 +0000 Subject: [PATCH 074/871] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index f0c97acb1f5..617e54a375e 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 2148f29a40f44f387b2cfbd9d3496bf9bc0b7e8d Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 8 Jun 2023 20:29:27 -0400 Subject: [PATCH 075/871] More accurate DNS resolve for the keeper connection --- src/Common/ZooKeeper/ZooKeeper.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..e078470476a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -13,6 +13,7 @@ #include #include #include "Common/ZooKeeper/IKeeper.h" +#include #include #include #include @@ -80,8 +81,12 @@ void ZooKeeper::init(ZooKeeperArgs args_) if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + /// We want to resolve all hosts without DNS cache for keeper connection. + Coordination::DNSResolver::instance().removeHostFromCache(host_string); + + auto address = Coordination::DNSResolver::instance().resolveAddress(host_string); + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, address.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{address, secure}); } catch (const Poco::Net::HostNotFoundException & e) { From bbf0548007432dc5482cd28fda4c31e57dd5c24f Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 02:48:28 +0000 Subject: [PATCH 076/871] fix test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 617e54a375e..9a8e22aa26f 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 6ad6c6afa3bdf1cd95e1454bad9e7eb75db7b0ab Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 04:13:16 +0000 Subject: [PATCH 077/871] fix --- src/Interpreters/Context.cpp | 2 +- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 823c3d678df..1b8c52ee06b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -783,7 +783,7 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + res += (obsolete_settings_count ? ", '" : "'") + setting.getName() + "'"; ++obsolete_settings_count; } } diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 9a8e22aa26f..5315c56bde8 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 0c05d5d7049..dcb296c61aa 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. +Obsolete settings [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From 404bfe773ef726b63e944b70a8b4253907637b8c Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 06:28:47 +0000 Subject: [PATCH 078/871] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 5315c56bde8..9be0eb6e399 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings" expect ":) " send -- "q\r" expect eof From e7d1dfb704caa283174823ba8ff59b6c10ae0e1d Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 08:30:07 +0000 Subject: [PATCH 079/871] fix --- tests/queries/0_stateless/01945_system_warnings.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 112baab614e..e44fe0ad6b5 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 1d8013074de7eb5a35460fef94bc82b8872531c3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 13 Jun 2023 18:47:02 +0200 Subject: [PATCH 080/871] fix style & black --- src/Storages/HDFS/StorageHDFS.cpp | 6 ++++-- src/Storages/StorageFile.cpp | 6 ++++-- tests/integration/test_storage_hdfs/test.py | 13 ++++++++++--- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 7e836c028a0..2503b0e60f2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -154,12 +154,14 @@ namespace const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 93228cf4d39..dc7cfe0de29 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -170,12 +170,14 @@ void listFilesWithRegexpMatchingImpl( /// slashes_in_glob counter is a upper-bound estimate of recursion depth /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index dde3dd257b1..789e8e11035 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -88,10 +88,17 @@ def test_read_write_storage_with_globs(started_cluster): def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api for i in ["1", "2"]: - hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") - assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + hdfs_api.write_data( + f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n" + ) + assert ( + hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") + == f"File{i}\t{i}{i}\n" + ) - r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + r = node1.query( + "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)" + ) assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") From 906db0318dee9d08a8af603ab0400143578e4f3d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Jun 2023 19:45:43 +0000 Subject: [PATCH 081/871] 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 3a23127e2fd..c53bed2007a 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 c38b3c79026..b6b45c664f9 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 10f8bbfd392..00000000000 --- 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 3b13b208eb5..00000000000 --- 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 44e0be8e356..00000000000 --- 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 8ef1273c855..00000000000 --- 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 082/871] 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 b3583e156ad..b5d52acca49 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 2b824ce91bd..96512fbbe23 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 083/871] 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 df25a9a3825..9548bf33b7b 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 084/871] 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 9548bf33b7b..17abc3d161d 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 085/871] 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 b5d52acca49..71d5885058a 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 96512fbbe23..df50d900919 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 086/871] 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 ec7bdd10196..fdcba5d4a4b 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 80c3535ef4e..53b6163ba87 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 02cf8a1bade3898196d7a760cb75ae10af3f2ecf Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 15 Jun 2023 14:09:38 +0300 Subject: [PATCH 087/871] Update comment StorageFile.cpp --- src/Storages/StorageFile.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 41ddd6b5780..4fbea89baa2 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -100,7 +100,14 @@ void listFilesWithRegexpMatchingImpl( std::vector & result, bool recursive = false); -/// When `{...}` has any `/`s, it must be processed in a different way +/* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by listFilesWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * listFilesWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. +*/ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const std::string & processed_suffix, const std::string & suffix_with_globs, @@ -111,14 +118,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /* - * When `{...}` has any `/`s, it must be processed in a different way: - * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, - * LSWithFoldedRegexpMatching is in charge from now on. - * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. - * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. - * StorageHDFS.cpp has the same logic. - */ if (!max_depth) return; From 9a25958be831d6068597703cef97238ab76ee325 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 15 Jun 2023 13:49:49 +0000 Subject: [PATCH 088/871] Add HTTP header filtering --- programs/server/Server.cpp | 2 + programs/server/config.xml | 8 +++ src/Common/HTTPHeaderFilter.cpp | 56 +++++++++++++++++++ src/Common/HTTPHeaderFilter.h | 27 +++++++++ src/Interpreters/Context.cpp | 16 +++++- src/Interpreters/Context.h | 5 ++ src/Storages/StorageS3.cpp | 1 + src/Storages/StorageS3Cluster.cpp | 2 + src/Storages/StorageURL.cpp | 1 + src/Storages/StorageURLCluster.cpp | 1 + tests/config/config.d/forbidden_headers.xml | 6 ++ tests/config/install.sh | 1 + .../02752_forbidden_headers.reference | 0 .../0_stateless/02752_forbidden_headers.sql | 18 ++++++ 14 files changed, 142 insertions(+), 2 deletions(-) create mode 100644 src/Common/HTTPHeaderFilter.cpp create mode 100644 src/Common/HTTPHeaderFilter.h create mode 100644 tests/config/config.d/forbidden_headers.xml create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.reference create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d0fc8aca5e8..6490eb6e3f5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -887,6 +887,7 @@ try #endif global_context->setRemoteHostFilter(config()); + global_context->setHTTPHeaderFilter(config()); std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); fs::path path = path_str; @@ -1184,6 +1185,7 @@ try } global_context->setRemoteHostFilter(*config); + global_context->setHTTPHeaderFilter(*config); global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop); global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop); diff --git a/programs/server/config.xml b/programs/server/config.xml index d18b4cb2ac9..f5013104630 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1006,6 +1006,14 @@ --> + + + - - - s3 s3_disk/ @@ -10,7 +9,6 @@ clickhouse 20000 - cache s3_disk @@ -88,13 +86,6 @@ - - -
- s3_cache_small -
-
-
From 665b6d43342f418a9c32bc31d1a969b53766fc96 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 5 Jul 2023 14:50:15 +0200 Subject: [PATCH 202/871] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 526e674a154..636b7a9747d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -761,6 +761,7 @@ Rollup RowBinary RowBinaryWithNames RowBinaryWithNamesAndTypes +RowBinaryWithDefaults Runtime SATA SELECTs @@ -2118,6 +2119,7 @@ rowNumberInBlock rowbinary rowbinarywithnames rowbinarywithnamesandtypes +rowbinarywithdefaults rsync rsyslog runnable From baee73fd96d1b1974ac7ec637c3b22c4f63a27a4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:11:25 +0200 Subject: [PATCH 203/871] 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 | 137 ++++++++++++------ 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, 253 insertions(+), 81 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 d2d8a0d07fb..0a311fa4737 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 e9ae6d8d937..d13378dcd65 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -102,6 +102,14 @@ private: 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 @@ private: 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 bb98e2bd3bb..4ba793d858d 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 0beb4492aef..84ecb1fc909 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 b262d88db57..c0d36836444 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -549,15 +549,15 @@ public: /** * 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 @@ public: /// 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 7424a248491..fc8f599a06e 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 dc24327712c..60c3999f87a 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 d7166b4a3b9..0e381654db0 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 d021667f771..9c05afd5284 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 8f089a4d580..db3cde93be5 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -92,7 +92,7 @@ public: 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 c46192ab43b..608f65cfeff 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 f45286341cf..547f61a012b 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -135,7 +135,7 @@ public: 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 14b7fc15af2..b243225adb3 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -139,7 +139,7 @@ public: 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 7f282b6c0e6..94727a5495c 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,39 +4828,9 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } -void StorageReplicatedMergeTree::partialShutdown() +void StorageReplicatedMergeTree::flushAndPrepareForShutdown() { - ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); - - partial_shutdown_called = true; - partial_shutdown_event.set(); - queue.notifySubscribersOnPartialShutdown(); - replica_is_active_node = nullptr; - - LOG_TRACE(log, "Waiting for threads to finish"); - merge_selecting_task->deactivate(); - queue_updating_task->deactivate(); - mutations_updating_task->deactivate(); - mutations_finalizing_task->deactivate(); - - cleanup_thread.stop(); - async_block_ids_cache.stop(); - part_check_thread.stop(); - - /// Stop queue processing - { - auto fetch_lock = fetcher.blocker.cancel(); - auto merge_lock = merger_mutator.merges_blocker.cancel(); - auto move_lock = parts_mover.moves_blocker.cancel(); - background_operations_assignee.finish(); - } - - LOG_TRACE(log, "Threads finished"); -} - -void StorageReplicatedMergeTree::shutdown() -{ - if (shutdown_called.exchange(true)) + if (shutdown_prepared_called.exchange(true)) return; session_expired_callback_handler.reset(); @@ -4860,6 +4859,58 @@ void StorageReplicatedMergeTree::shutdown() } 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(); + 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(); + queue_updating_task->deactivate(); + mutations_updating_task->deactivate(); + mutations_finalizing_task->deactivate(); + + cleanup_thread.stop(); + async_block_ids_cache.stop(); + part_check_thread.stop(); + + /// Stop queue processing + { + auto fetch_lock = fetcher.blocker.cancel(); + auto merge_lock = merger_mutator.merges_blocker.cancel(); + auto move_lock = parts_mover.moves_blocker.cancel(); + background_operations_assignee.finish(); + } + + LOG_TRACE(log, "Threads finished"); +} + +void StorageReplicatedMergeTree::shutdown() +{ + if (shutdown_called.exchange(true)) + return; + + if (!shutdown_prepared_called.load()) + 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()); + + 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 4661f0a56da..104062def4b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -113,7 +113,10 @@ public: 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 @@ private: 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 @@ private: */ 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 26cbe1f0233..5df050d1d0d 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -79,11 +79,11 @@ public: 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 00000000000..e5a0d9b4834 --- /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 00000000000..8ff3bdf9a2f --- /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 00000000000..75f0921646e --- /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 204/871] 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 75f0921646e..e3a2e7a0271 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 @@ import time 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 test_shutdown_and_wait(start_cluster): 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 205/871] Fixes and comments --- programs/server/Server.cpp | 16 ++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++ src/Storages/StorageReplicatedMergeTree.h | 30 ++++++++++++++++++--- 3 files changed, 44 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0a311fa4737..58cf3e5d210 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 94727a5495c..2f165a056a3 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 104062def4b..baa5af824b4 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -112,11 +112,35 @@ public: bool need_check_structure); void startup() 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 flushAndPrepareForShutdown() 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 206/871] 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 2f165a056a3..e5abf63a72d 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 baa5af824b4..a1a0717ca64 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -374,6 +374,8 @@ public: 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 207/871] 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 e5abf63a72d..114465df496 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 a1a0717ca64..656e8df6ccb 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -121,7 +121,7 @@ public: 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 @@ public: /// 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 e3a2e7a0271..a2a4ec92cf7 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 test_shutdown_and_wait(start_cluster): # 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 2ab6c599a234d31c2f59e1aaa35298c1274390b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 Jul 2023 23:31:44 +0200 Subject: [PATCH 208/871] Fix tests --- .../0_stateless/02240_filesystem_query_cache.sql | 15 +++++++++++++++ .../0_stateless/02286_drop_filesystem_cache.sh | 5 ----- .../0_stateless/02344_describe_cache.reference | 2 +- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 760ec1baa87..02cf54b0caa 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -9,6 +9,21 @@ SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; SYSTEM DROP FILESYSTEM CACHE; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh index 091bca10bcf..1e1841862e9 100755 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh @@ -67,9 +67,4 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do ON data_paths.cache_path = caches.cache_path" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_022862" - - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_022862 (key UInt32, value String) - Engine=MergeTree() - ORDER BY key - SETTINGS storage_policy='${STORAGE_POLICY}_2', min_bytes_for_wide_part = 10485760" done diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 5c3d47d87f6..da84cdabf79 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1 +1 @@ -134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 From 45db928e4e31aae6a6d7e8e6b35e0a5a3768375c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 02:52:55 +0200 Subject: [PATCH 209/871] 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 8736b1fce7f..695d1b7d63c 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 5b4cd34141c..907c7570ce2 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 dee71d2e2f8cdd6be4a82f26e7af9b8a75453091 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 13:16:31 +0000 Subject: [PATCH 210/871] Add first version of hasSubsequence() --- src/Functions/HasSubsequenceImpl.h | 131 ++++++++++++++++++ src/Functions/hasSubsequence.cpp | 29 ++++ .../hasSubsequenceCaseInsensitive.cpp | 28 ++++ src/Functions/like.cpp | 1 - .../02809_has_subsequence.reference | 16 +++ .../0_stateless/02809_has_subsequence.sql | 19 +++ 6 files changed, 223 insertions(+), 1 deletion(-) create mode 100644 src/Functions/HasSubsequenceImpl.h create mode 100644 src/Functions/hasSubsequence.cpp create mode 100644 src/Functions/hasSubsequenceCaseInsensitive.cpp create mode 100644 tests/queries/0_stateless/02809_has_subsequence.reference create mode 100644 tests/queries/0_stateless/02809_has_subsequence.sql diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h new file mode 100644 index 00000000000..3a29ef68b0b --- /dev/null +++ b/src/Functions/HasSubsequenceImpl.h @@ -0,0 +1,131 @@ +#pragma once + + +namespace DB +{ +namespace +{ + +template +struct HasSubsequenceImpl +{ + using ResultType = UInt8; + + static constexpr bool use_default_implementation_for_constants = false; + static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; + + static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + + /// Find one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & /*haystack_data*/, + const ColumnString::Offsets & /*haystack_offsets*/, + const std::string & /*needle*/, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + [[maybe_unused]] ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + /// Search each time for a different single substring inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = impl(haystack, haystack_size, needle, needle_size); + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Find many substrings in single string. + static void constantVector( + const String & /*haystack*/, + const ColumnString::Chars & /*needle_data*/, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + + static void constantConstant( + std::string haystack, + std::string needle, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); + + UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + + for (size_t i = 0; i < size; ++i) + { + res[i] = result; + } + } + template + static void vectorFixedConstant(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } + + template + static void vectorFixedVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } +}; + +} + +} diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp new file mode 100644 index 00000000000..da2aaddcf50 --- /dev/null +++ b/src/Functions/hasSubsequence.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveASCII +{ + static void toLowerIfNeed(std::string & /*s*/) { } +}; + +struct NameHasSubsequence +{ + static constexpr auto name = "hasSubsequence"; +}; + +using FunctionHasSubsequence = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequence) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp new file mode 100644 index 00000000000..f5c13a7cf8c --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveASCII +{ + static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitive +{ + static constexpr auto name = "hasSubsequenceCaseInsensitive"; +}; + +using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index 3a3345051d4..5a86e37a92d 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -1,4 +1,3 @@ -#include "FunctionsStringSearch.h" #include "FunctionFactory.h" #include "like.h" diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference new file mode 100644 index 00000000000..827caa105d0 --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -0,0 +1,16 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +1 +1 +1 +0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql new file mode 100644 index 00000000000..63ffb49dc54 --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -0,0 +1,19 @@ +select hasSubsequence('garbage', ''); +select hasSubsequence('garbage', 'g'); +select hasSubsequence('garbage', 'a'); +select hasSubsequence('garbage', 'e'); +select hasSubsequence('garbage', 'gr'); +select hasSubsequence('garbage', 'ab'); +select hasSubsequence('garbage', 'be'); +select hasSubsequence('garbage', 'arg'); +select hasSubsequence('garbage', 'garbage'); + +select hasSubsequence('garbage', 'garbage1'); +select hasSubsequence('garbage', 'arbw'); +select hasSubsequence('garbage', 'ARG'); + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + +select hasSubsequence(materialize('garbage'), materialize('')); +select hasSubsequence(materialize('garbage'), materialize('arg')); +select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file From 810d1ee0694cc769170f4b08c58aa4c2c5b0807a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Jul 2023 13:48:57 +0000 Subject: [PATCH 211/871] 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 b7b1b0b29a6..00888cfa5e9 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -85,7 +85,7 @@ private: 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 df77994c3d5..2db8bd6c59c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -50,7 +50,7 @@ private: 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 5ab20c796ea..bb52e2aa516 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -67,7 +67,7 @@ protected: 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 f8c9a39eedf..65ea87479a3 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -66,7 +66,7 @@ private: 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 98561e72e61..7097ea3ac08 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -52,7 +52,7 @@ private: 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 4495680f5b2..f61dc3fbc78 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -202,7 +202,7 @@ private: 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 a14c51f8b9f..dc14edf2099 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -273,7 +273,7 @@ private: 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 d540a24fa70..8f8d44ec088 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -96,7 +96,7 @@ private: 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 b9be01cf2ae..e583d2e30b7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -301,18 +301,26 @@ public: 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 67e2dee7e2ea926d6a0a6ab35b31b2515f518426 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 14:29:58 +0000 Subject: [PATCH 212/871] Allow SETTINGS before FORMAT in DESCRIBE TABLE query --- src/Parsers/ParserDescribeTableQuery.cpp | 20 +++++++++++++++---- src/Parsers/ParserQueryWithOutput.cpp | 2 +- src/Parsers/ParserTablePropertiesQuery.cpp | 2 -- src/Storages/StorageDistributed.cpp | 1 - src/Storages/getStructureOfRemoteTable.cpp | 1 - .../02789_describe_table_settings.reference | 10 ++++++++++ .../02789_describe_table_settings.sql | 3 +++ 7 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.reference create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.sql diff --git a/src/Parsers/ParserDescribeTableQuery.cpp b/src/Parsers/ParserDescribeTableQuery.cpp index ad6d2c5bcc6..fcfc4799dbe 100644 --- a/src/Parsers/ParserDescribeTableQuery.cpp +++ b/src/Parsers/ParserDescribeTableQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -16,8 +17,10 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ParserKeyword s_describe("DESCRIBE"); ParserKeyword s_desc("DESC"); ParserKeyword s_table("TABLE"); + ParserKeyword s_settings("SETTINGS"); ParserToken s_dot(TokenType::Dot); ParserIdentifier name_p; + ParserSetQuery parser_settings(true); ASTPtr database; ASTPtr table; @@ -29,12 +32,21 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex s_table.ignore(pos, expected); - ASTPtr table_expression; - if (!ParserTableExpression().parse(pos, table_expression, expected)) + if (!ParserTableExpression().parse(pos, query->table_expression, expected)) return false; - query->children.push_back(std::move(table_expression)); - query->table_expression = query->children.back(); + /// For compatibility with SELECTs, where SETTINGS can be in front of FORMAT + ASTPtr settings; + if (s_settings.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, query->settings_ast, expected)) + return false; + } + + query->children.push_back(query->table_expression); + + if (query->settings_ast) + query->children.push_back(query->settings_ast); node = query; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..5dc713ca8c6 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -150,7 +150,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec // SETTINGS key1 = value1, key2 = value2, ... ParserKeyword s_settings("SETTINGS"); - if (s_settings.ignore(pos, expected)) + if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) { ParserSetQuery parser_settings(true); if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) diff --git a/src/Parsers/ParserTablePropertiesQuery.cpp b/src/Parsers/ParserTablePropertiesQuery.cpp index b73ce8de359..94f264fcc89 100644 --- a/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/src/Parsers/ParserTablePropertiesQuery.cpp @@ -14,8 +14,6 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { ParserKeyword s_exists("EXISTS"); ParserKeyword s_temporary("TEMPORARY"); - ParserKeyword s_describe("DESCRIBE"); - ParserKeyword s_desc("DESC"); ParserKeyword s_show("SHOW"); ParserKeyword s_create("CREATE"); ParserKeyword s_database("DATABASE"); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b91ad0b963a..b6359bbb251 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -60,7 +60,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f4..ec8f27feeda 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/tests/queries/0_stateless/02789_describe_table_settings.reference b/tests/queries/0_stateless/02789_describe_table_settings.reference new file mode 100644 index 00000000000..c2bf9219f4d --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.reference @@ -0,0 +1,10 @@ +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" diff --git a/tests/queries/0_stateless/02789_describe_table_settings.sql b/tests/queries/0_stateless/02789_describe_table_settings.sql new file mode 100644 index 00000000000..64b5b21fea8 --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.sql @@ -0,0 +1,3 @@ +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 FORMAT CSV; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 SETTINGS max_threads=0; -- { clientError SYNTAX_ERROR } From 9a295eca46fea2c88d1c1767fc4625b31c999572 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jul 2023 14:28:50 +0000 Subject: [PATCH 213/871] Incorporate review feedback --- docs/en/sql-reference/statements/show.md | 24 ++--- .../InterpreterShowIndexesQuery.cpp | 4 +- .../0_stateless/02724_show_indexes.reference | 88 +++++++++---------- 3 files changed, 58 insertions(+), 58 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 1a1e4dbd2c7..1c399d2072b 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -289,18 +289,18 @@ The statement produces a result table with the following structure: - table - The name of the table. (String) - non_unique - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8) - key_name - The name of the index, `PRIMARY` if the index is a primary key index. (String) -- column_name - For a primary key index, the name of the column. For a data skipping index: '' (empty string), see field "expression". (String) - seq_in_index - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8) +- column_name - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String) - collation - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String)) - cardinality - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64) - sub_part - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String)) - packed - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String)) - null - Currently unused - index_type - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String) -- comment - Additional information about the index, currently always `` (empty string). (String) -- index_comment - `` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) +- comment - Additional information about the index, currently always `''` (empty string). (String) +- index_comment - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) - visible - If the index is visible to the optimizer, always `YES`. (String) -- expression - For a data skipping index, the index expression. For a primary key index: '' (empty string). (String) +- expression - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String) **Examples** @@ -313,14 +313,14 @@ SHOW INDEX FROM 'tbl' Result: ``` text -┌─table─┬─non_unique─┬─key_name─┬─column_name─┬─seq_in_index─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ -│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ -│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ -│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ -│ tbl │ 1 │ PRIMARY │ c │ 1 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ PRIMARY │ a │ 2 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ -└───────┴────────────┴──────────┴─────────────┴──────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ +┌─table─┬─non_unique─┬─key_name─┬─seq_in_index─┬─column_name─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ +│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ +│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ +│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ +│ tbl │ 1 │ PRIMARY │ 1 │ c │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ PRIMARY │ 2 │ a │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ +└───────┴────────────┴──────────┴──────────────┴─────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ ``` **See also** diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 35f32a79310..149420006fb 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -42,8 +42,8 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - arrayJoin(splitByString(', ', primary_key)) AS column_name, row_number() over (order by column_name) AS seq_in_index, + arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, NULL AS sub_part, @@ -63,8 +63,8 @@ FROM ( table AS table, 1 AS non_unique, name AS key_name, - '' AS column_name, 1 AS seq_in_index, + '' AS column_name, NULL AS collation, 0 AS cardinality, NULL AS sub_part, diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index cee0598d625..e41f2521f5c 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -1,51 +1,51 @@ --- Aliases of SHOW INDEX -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names -$4@^7 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -NULL 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES +$4@^7 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES --- Original table -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES --- Short form -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES From 9ae0dc730c586a37f8fdbbd880267ec11c2c8e51 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:15:26 +0200 Subject: [PATCH 214/871] 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 82a88272df9..07e6e7dd0ec 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 0e381654db0..e43cc879e93 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 9e99baab4c3..01a877a07e5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -36,6 +36,7 @@ public: 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 114465df496..88bd788b9ef 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 656e8df6ccb..811a8524064 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -118,7 +118,7 @@ public: /// 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 @@ public: 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 @@ private: 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 @@ private: */ 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 00000000000..b23dbdc2607 --- /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 50f2627d37c..33d5c99202e 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 215/871] 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 b23dbdc2607..504841296a8 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 216/871] 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 88bd788b9ef..0f5a52b275c 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 7255c35edcefe03a39ad7bcf460d9dca5670ca3b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 19:43:37 +0000 Subject: [PATCH 217/871] Add more tests --- .../functions/string-search-functions.md | 50 +++++++++++++ .../functions/string-search-functions.md | 52 +++++++++++++ src/Functions/HasSubsequenceImpl.h | 74 ++++++++++++------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 28 +++++++ src/Functions/hasSubsequenceUTF8.cpp | 29 ++++++++ .../02809_has_subsequence.reference | 13 +++- .../0_stateless/02809_has_subsequence.sql | 20 ++++- 9 files changed, 237 insertions(+), 33 deletions(-) create mode 100644 src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp create mode 100644 src/Functions/hasSubsequenceUTF8.cpp diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 3d8f89f7295..04ad6474310 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -631,3 +631,53 @@ Result: │ 100 │ 200 │ 100-200 │ 100 │ └──────────────────────────────────────────────┴──────────────────────────────────────────────┴──────────────────────────────────────────────┴───────────────────────────────────────────┘ ``` + +## hasSubsequence + +Returns 1 if needle is a subsequence of haystack, or 0 otherwise. +A subsequence of a string is a sequence that can be derived from the given string by deleting zero or more elements without changing the order of the remaining elements. + + +**Syntax** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Arguments** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). + +**Returned values** + +- 1, if needle is a subsequence of haystack. +- 0, otherwise. + +Type: `UInt8`. + +**Examples** + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Result: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + +## hasSubsequenceCaseInsensitive + +Like [hasSubsequence](#hasSubsequence) but searches case-insensitively. + +## hasSubsequenceUTF8 + +Like [hasSubsequence](#hasSubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. + +## hasSubsequenceCaseInsensitiveUTF8 + +Like [hasSubsequenceUTF8](#hasSubsequenceUTF8) but searches case-insensitively. \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index ea4f90d4f66..21989e882b6 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -801,3 +801,55 @@ SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв'); │ 3 │ └────────────────────────────────────────────────────────────┘ ``` + +## hasSubsequence(haystack, needle) {#hasSubsequence} + +Возвращает 1 если needle является подпоследовательностью haystack, иначе 0. + + +**Синтаксис** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Аргументы** + +- `haystack` — строка, по которой выполняется поиск. [Строка](../syntax.md#syntax-string-literal). +- `needle` — подстрока, которую необходимо найти. [Строка](../syntax.md#syntax-string-literal). + +**Возвращаемые значения** + +- 1, если +- 0, если подстрока не найдена. + +Тип: `UInt8`. + +**Примеры** + +Запрос: + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Результат: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + + +## hasSubsequenceCaseInsensitive + +Такая же, как и [hasSubsequence](#hasSubsequence), но работает без учета регистра. + +## hasSubsequenceUTF8 + +Такая же, как и [hasSubsequence](#hasSubsequence) при допущении что `haystack` и `needle` содержат набор кодовых точек, представляющий текст в кодировке UTF-8. + +## hasSubsequenceCaseInsensitiveUTF8 + +Такая же, как и [hasSubsequenceUTF8](#hasSubsequenceUTF8), но работает без учета регистра. diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index 3a29ef68b0b..bcb8e8e99e6 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,11 +1,8 @@ #pragma once - - namespace DB { namespace { - template struct HasSubsequenceImpl { @@ -17,23 +14,31 @@ struct HasSubsequenceImpl static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} - /// Find one substring in many strings. static void vectorConstant( - const ColumnString::Chars & /*haystack_data*/, - const ColumnString::Offsets & /*haystack_offsets*/, - const std::string & /*needle*/, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const String & needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, [[maybe_unused]] ColumnUInt8 * /*res_null*/) { - size_t size = res.size(); - for (size_t i = 0; i < size; ++i) + if (needle.empty()) { - res[i] = 0; + for (auto & r : res) + r = 1; + return; + } + + ColumnString::Offset prev_haystack_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); + prev_haystack_offset = haystack_offsets[i]; } } - /// Search each time for a different single substring inside each time different string. static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, @@ -61,7 +66,7 @@ struct HasSubsequenceImpl { const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = impl(haystack, haystack_size, needle, needle_size); + res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); } prev_haystack_offset = haystack_offsets[i]; @@ -69,35 +74,38 @@ struct HasSubsequenceImpl } } - /// Find many substrings in single string. static void constantVector( - const String & /*haystack*/, - const ColumnString::Chars & /*needle_data*/, + const String & haystack, + const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) { + ColumnString::Offset prev_needle_offset = 0; + size_t size = needle_offsets.size(); for (size_t i = 0; i < size; ++i) { - res[i] = 0; + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); + } + prev_needle_offset = needle_offsets[i]; } } - static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) - { - size_t j = 0; - for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) - if (needle[j] == haystack[i]) - ++j; - return j == needle_size; - } - static void constantConstant( - std::string haystack, - std::string needle, + String haystack, + String needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) @@ -106,13 +114,23 @@ struct HasSubsequenceImpl Impl::toLowerIfNeed(haystack); Impl::toLowerIfNeed(needle); - UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); for (size_t i = 0; i < size; ++i) { res[i] = result; } } + + static UInt8 hasSubsequence(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + template static void vectorFixedConstant(Args &&...) { diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index da2aaddcf50..bb1f295cee4 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -10,7 +10,7 @@ namespace struct HasSubsequenceCaseSensitiveASCII { - static void toLowerIfNeed(std::string & /*s*/) { } + static void toLowerIfNeed(String & /*s*/) { } }; struct NameHasSubsequence diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index f5c13a7cf8c..fe50ada9be9 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -9,7 +9,7 @@ namespace struct HasSubsequenceCaseInsensitiveASCII { - static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } }; struct NameHasSubsequenceCaseInsensitive diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..2908c284a25 --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveUTF8 +{ + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitiveUTF8 +{ + static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; +}; + +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp new file mode 100644 index 00000000000..c0811de6575 --- /dev/null +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveUTF8 +{ + static void toLowerIfNeed(String & /*s*/) { } +}; + +struct NameHasSubsequenceUTF8 +{ + static constexpr auto name = "hasSubsequenceUTF8"; +}; + +using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 827caa105d0..d12c0ba9fb3 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -1,3 +1,4 @@ +hasSubsequence / const / const 1 1 1 @@ -10,7 +11,17 @@ 0 0 0 +hasSubsequence / const / string 1 1 +0 +hasSubsequence / string / const +1 +1 +0 +hasSubsequence / string / string +1 +1 +0 +hasSubsequenceCaseInsensitive / const / const 1 -0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index 63ffb49dc54..64f3fd8dc77 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -1,3 +1,4 @@ +select 'hasSubsequence / const / const'; select hasSubsequence('garbage', ''); select hasSubsequence('garbage', 'g'); select hasSubsequence('garbage', 'a'); @@ -12,8 +13,23 @@ select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); select hasSubsequence('garbage', 'ARG'); -select hasSubsequenceCaseInsensitive('garbage', 'ARG'); +select 'hasSubsequence / const / string'; +select hasSubsequence('garbage', materialize('')); +select hasSubsequence('garbage', materialize('arg')); +select hasSubsequence('garbage', materialize('arbw')); + +select 'hasSubsequence / string / const'; +select hasSubsequence(materialize('garbage'), ''); +select hasSubsequence(materialize('garbage'), 'arg'); +select hasSubsequence(materialize('garbage'), 'arbw'); + +select 'hasSubsequence / string / string'; select hasSubsequence(materialize('garbage'), materialize('')); select hasSubsequence(materialize('garbage'), materialize('arg')); -select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file +select hasSubsequence(materialize('garbage'), materialize('garbage1')); + +select 'hasSubsequenceCaseInsensitive / const / const'; + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + From 8b6376005a730b9ae461d3fe93a55e51cd494181 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 20 Apr 2023 13:26:02 +0000 Subject: [PATCH 218/871] "reconfig" support for CH Keeper --- base/base/find_symbols.h | 4 +- base/base/move_extend.h | 9 + contrib/NuRaft | 2 +- programs/keeper/CMakeLists.txt | 2 + src/Common/ProfileEvents.cpp | 2 + src/Common/ZooKeeper/IKeeper.cpp | 1 + src/Common/ZooKeeper/IKeeper.h | 41 +++- src/Common/ZooKeeper/TestKeeper.cpp | 56 ++++- src/Common/ZooKeeper/TestKeeper.h | 7 + src/Common/ZooKeeper/ZooKeeper.cpp | 32 ++- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 44 +++- src/Common/ZooKeeper/ZooKeeperCommon.h | 29 +++ src/Common/ZooKeeper/ZooKeeperConstants.cpp | 50 +--- src/Common/ZooKeeper/ZooKeeperConstants.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 28 ++- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 + src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.h | 10 +- src/Coordination/KeeperDispatcher.cpp | 103 +++++--- src/Coordination/KeeperDispatcher.h | 16 +- src/Coordination/KeeperReconfiguration.cpp | 92 +++++++ src/Coordination/KeeperReconfiguration.h | 10 + src/Coordination/KeeperServer.cpp | 232 ++++++++---------- src/Coordination/KeeperServer.h | 23 +- src/Coordination/KeeperStateMachine.cpp | 124 ++++++++-- src/Coordination/KeeperStateMachine.h | 16 +- src/Coordination/KeeperStateManager.cpp | 15 +- src/Coordination/KeeperStateManager.h | 32 +-- src/Coordination/KeeperStorage.cpp | 20 +- src/Coordination/RaftServerConfig.cpp | 96 ++++++++ src/Coordination/RaftServerConfig.h | 78 ++++++ src/Coordination/tests/gtest_coordination.cpp | 51 +++- src/Interpreters/ZooKeeperLog.cpp | 1 + src/Storages/DataLakes/HudiMetadataParser.cpp | 3 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 8 +- src/Storages/StorageReplicatedMergeTree.cpp | 16 +- tests/integration/helpers/keeper_utils.py | 33 ++- .../test_keeper_nodes_move/test.py | 5 - .../test_keeper_reconfig_add/__init__.py | 0 .../configs/keeper1.xml | 20 ++ .../configs/keeper2.xml | 21 ++ .../configs/keeper3.xml | 22 ++ .../test_keeper_reconfig_add/test.py | 155 ++++++++++++ .../test_keeper_reconfig_remove/__init__.py | 0 .../configs/keeper1.xml | 37 +++ .../configs/keeper2.xml | 37 +++ .../configs/keeper3.xml | 37 +++ .../test_keeper_reconfig_remove/test.py | 145 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 47 ++++ .../configs/keeper2.xml | 47 ++++ .../configs/keeper3.xml | 47 ++++ .../configs/keeper4.xml | 47 ++++ .../configs/keeper5.xml | 47 ++++ .../test_keeper_reconfig_remove_many/test.py | 149 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 127 ++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 120 +++++++++ utils/keeper-data-dumper/main.cpp | 2 +- 70 files changed, 2309 insertions(+), 367 deletions(-) create mode 100644 base/base/move_extend.h create mode 100644 src/Coordination/KeeperReconfiguration.cpp create mode 100644 src/Coordination/KeeperReconfiguration.h create mode 100644 src/Coordination/RaftServerConfig.cpp create mode 100644 src/Coordination/RaftServerConfig.h create mode 100644 tests/integration/test_keeper_reconfig_add/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_add/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index 83232669c04..f7d24ccfc11 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline void splitInto(To & to, const std::string & what, bool token_compress = false) +inline To& splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); @@ -464,4 +464,6 @@ inline void splitInto(To & to, const std::string & what, bool token_compress = f else pos = delimiter_or_end; } + + return to; } diff --git a/base/base/move_extend.h b/base/base/move_extend.h new file mode 100644 index 00000000000..6e5b16e037c --- /dev/null +++ b/base/base/move_extend.h @@ -0,0 +1,9 @@ +#pragma once + +/// Extend @p to by moving elements from @p from to @p to end +/// @return @p to iterator to first of moved elements. +template +typename To::iterator moveExtend(To & to, From && from) +{ + return to.insert(to.end(), std::make_move_iterator(from.begin()), std::make_move_iterator(from.end())); +} diff --git a/contrib/NuRaft b/contrib/NuRaft index 491eaf592d9..eb1572129c7 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 491eaf592d950e0e37accbe8b3f217e068c9fecf +Subproject commit eb1572129c71beb2156dcdaadc3fb136954aed96 diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 18bdc8f317c..20cab03dec2 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -34,6 +34,8 @@ add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs) if (BUILD_STANDALONE_KEEPER) # Straight list of all required sources set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperReconfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/RaftServerConfig.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ACLMap.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Changelog.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/CoordinationSettings.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0838e0366df..8e3ec4f9e65 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -125,6 +125,7 @@ M(ZooKeeperMulti, "Number of 'multi' requests to ZooKeeper (compound transactions).") \ M(ZooKeeperCheck, "Number of 'check' requests to ZooKeeper. Usually they don't make sense in isolation, only as part of a complex transaction.") \ M(ZooKeeperSync, "Number of 'sync' requests to ZooKeeper. These requests are rarely needed or usable.") \ + M(ZooKeeperReconfig, "Number of 'reconfig' requests to ZooKeeper.") \ M(ZooKeeperClose, "Number of times connection with ZooKeeper has been closed voluntary.") \ M(ZooKeeperWatchResponse, "Number of times watch notification has been received from ZooKeeper.") \ M(ZooKeeperUserExceptions, "Number of exceptions while working with ZooKeeper related to the data (no node, bad version or similar).") \ @@ -499,6 +500,7 @@ The server successfully detected this situation and will download merged part fr M(KeeperCreateRequest, "Number of create requests")\ M(KeeperRemoveRequest, "Number of remove requests")\ M(KeeperSetRequest, "Number of set requests")\ + M(KeeperReconfigRequest, "Number of reconfig requests")\ M(KeeperCheckRequest, "Number of check requests")\ M(KeeperMultiRequest, "Number of multi requests")\ M(KeeperMultiReadRequest, "Number of multi read requests")\ diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index f0a07241735..50160279506 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,6 +110,7 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; + case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 2703c1079c0..20ce2a748e6 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,6 +82,7 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state + ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -350,6 +351,29 @@ struct SyncResponse : virtual Response size_t bytesSize() const override { return path.size(); } }; +struct ReconfigRequest : virtual Request +{ + String joining; + String leaving; + String new_members; + int32_t version; + + String getPath() const final { return keeper_config_path; } + + size_t bytesSize() const final + { + return joining.size() + leaving.size() + new_members.size() + sizeof(version); + } +}; + +struct ReconfigResponse : virtual Response +{ + String value; + Stat stat; + + size_t bytesSize() const override { return value.size() + sizeof(stat); } +}; + struct MultiRequest : virtual Request { Requests requests; @@ -395,9 +419,9 @@ using SetCallback = std::function; using ListCallback = std::function; using CheckCallback = std::function; using SyncCallback = std::function; +using ReconfigCallback = std::function; using MultiCallback = std::function; - /// For watches. enum State { @@ -526,6 +550,13 @@ public: const String & path, SyncCallback callback) = 0; + virtual void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) = 0; + virtual void multi( const Requests & requests, MultiCallback callback) = 0; @@ -539,3 +570,11 @@ public: }; } + +template <> struct fmt::formatter : fmt::formatter +{ + constexpr auto format(Coordination::Error code, auto& ctx) + { + return formatter::format(Coordination::errorMessage(code), ctx); + } +}; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index fe4cb83c78a..87c87c4fc92 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -3,12 +3,8 @@ #include #include #include - -#include -#include #include - namespace Coordination { @@ -147,6 +143,14 @@ struct TestKeeperSyncRequest final : SyncRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; +struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest +{ + TestKeeperReconfigRequest() = default; + explicit TestKeeperReconfigRequest(const ReconfigRequest & base) : ReconfigRequest(base) {} + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; +}; + struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { explicit TestKeeperMultiRequest(const Requests & generic_requests) @@ -226,15 +230,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai std::string path_created = path; if (is_sequential) - { - auto seq_num = it->second.seq_num; - - std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - seq_num_str.exceptions(std::ios::failbit); - seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - - path_created += seq_num_str.str(); - } + path_created += fmt::format("{:0>10}", it->second.seq_num); /// Increment sequential number even if node is not sequential ++it->second.seq_num; @@ -446,6 +442,17 @@ std::pair TestKeeperSyncRequest::process(TestKeeper::Containe return { std::make_shared(std::move(response)), {} }; } +std::pair TestKeeperReconfigRequest::process(TestKeeper::Container &, int64_t) const +{ + // In TestKeeper we assume data is stored on one server, so this is a dummy implementation to + // satisfy IKeeper interface. + // We can't even check the validity of input data, neither can we create the /keeper/config znode + // as we don't know the id of current "server". + ReconfigResponse response; + response.error = Error::ZOK; + return { std::make_shared(std::move(response)), {} }; +} + std::pair TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const { MultiResponse response; @@ -505,6 +512,7 @@ ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shar ResponsePtr TestKeeperListRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperSyncRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperReconfigRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared(); } @@ -828,6 +836,28 @@ void TestKeeper::sync( pushRequest(std::move(request_info)); } +void TestKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + TestKeeperReconfigRequest req; + req.joining = joining; + req.leaving = leaving; + req.new_members = new_members; + req.version = version; + + pushRequest({ + .request = std::make_shared(std::move(req)), + .callback = [callback](const Response & response) + { + callback(dynamic_cast(response)); + } + }); +} + void TestKeeper::multi( const Requests & requests, MultiCallback callback) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 9bbd018cfb1..8615ed0fb77 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -87,6 +87,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5dd7948276d..12b1d82133e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -75,13 +75,14 @@ void ZooKeeper::init(ZooKeeperArgs args_) auto & host_string = host.host; try { - bool secure = startsWith(host_string, "secure://"); + const bool secure = startsWith(host_string, "secure://"); if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + const Poco::Net::SocketAddress host_socket_addr{host_string}; + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, host_socket_addr.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, secure}); } catch (const Poco::Net::HostNotFoundException & e) { @@ -191,12 +192,7 @@ std::vector ZooKeeper::shuffleHosts() const shuffle_hosts.emplace_back(shuffle_host); } - ::sort( - shuffle_hosts.begin(), shuffle_hosts.end(), - [](const ShuffleHost & lhs, const ShuffleHost & rhs) - { - return ShuffleHost::compare(lhs, rhs); - }); + ::sort(shuffle_hosts.begin(), shuffle_hosts.end(), ShuffleHost::compare); return shuffle_hosts; } @@ -231,7 +227,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::List, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -298,7 +294,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Create, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -393,7 +389,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Remove, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -425,7 +421,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Exists, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -459,7 +455,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Get, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -531,7 +527,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Set, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -583,7 +579,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath())); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Multi, requests[0]->getPath())); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -617,7 +613,7 @@ Coordination::Error ZooKeeper::syncImpl(const std::string & path, std::string & if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Sync), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Sync, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -1229,7 +1225,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: if (!Coordination::isUserError(exception_code)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There are no failed OPs because '{}' is not valid response code for that", - std::string(Coordination::errorMessage(exception_code))); + exception_code); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is no failed OpResult"); } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 5031af38812..c24eecbafd8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -36,7 +36,7 @@ std::string ZooKeeperRequest::toString() const "OpNum = {}\n" "Additional info:\n{}", xid, - Coordination::toString(getOpNum()), + getOpNum(), toStringImpl()); } @@ -76,6 +76,41 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(joining, out); + Coordination::write(leaving, out); + Coordination::write(new_members, out); + Coordination::write(version, out); +} + +void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(joining, in); + Coordination::read(leaving, in); + Coordination::read(new_members, in); + Coordination::read(version, in); +} + +std::string ZooKeeperReconfigRequest::toStringImpl() const +{ + return fmt::format( + "joining = {}\nleaving = {}\nnew_members = {}\nversion = {}", + joining, leaving, new_members, version); +} + +void ZooKeeperReconfigResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(value, in); + Coordination::read(stat, in); +} + +void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(value, out); + Coordination::write(stat, out); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -664,6 +699,7 @@ ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTi ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -861,7 +897,8 @@ void ZooKeeperMultiResponse::fillLogElements(LogElements & elems, size_t idx) co void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator) { if (!op_num_to_request.try_emplace(op_num, creator).second) - throw Coordination::Exception("Request type " + toString(op_num) + " already registered", Coordination::Error::ZRUNTIMEINCONSISTENCY); + throw Coordination::Exception(Coordination::Error::ZRUNTIMEINCONSISTENCY, + "Request type {} already registered", op_num); } std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) @@ -916,7 +953,7 @@ ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); if (it == op_num_to_request.end()) - throw Exception("Unknown operation type " + toString(op_num), Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Unknown operation type {}", op_num); return it->second(); } @@ -960,6 +997,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 5f00698423e..131d19f1ca4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -117,6 +117,35 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse OpNum getOpNum() const override { return OpNum::Sync; } }; +struct ZooKeeperReconfigRequest final : ZooKeeperRequest +{ + String joining; + String leaving; + String new_members; + int64_t version; // kazoo sends a 64bit integer in this request + + String getPath() const override { return keeper_config_path; } + OpNum getOpNum() const override { return OpNum::Reconfig; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl() const override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } + + size_t bytesSize() const override + { + return ZooKeeperRequest::bytesSize() + joining.size() + leaving.size() + new_members.size() + + sizeof(version); + } +}; + +struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return OpNum::Reconfig; } +}; + struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 86f70ea547a..9bb9c7b0488 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -19,6 +19,7 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::Heartbeat), static_cast(OpNum::List), static_cast(OpNum::Check), + static_cast(OpNum::Reconfig), static_cast(OpNum::Multi), static_cast(OpNum::MultiRead), static_cast(OpNum::Auth), @@ -29,55 +30,6 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::CheckNotExists), }; -std::string toString(OpNum op_num) -{ - switch (op_num) - { - case OpNum::Close: - return "Close"; - case OpNum::Error: - return "Error"; - case OpNum::Create: - return "Create"; - case OpNum::Remove: - return "Remove"; - case OpNum::Exists: - return "Exists"; - case OpNum::Get: - return "Get"; - case OpNum::Set: - return "Set"; - case OpNum::SimpleList: - return "SimpleList"; - case OpNum::List: - return "List"; - case OpNum::Check: - return "Check"; - case OpNum::Multi: - return "Multi"; - case OpNum::MultiRead: - return "MultiRead"; - case OpNum::Sync: - return "Sync"; - case OpNum::Heartbeat: - return "Heartbeat"; - case OpNum::Auth: - return "Auth"; - case OpNum::SessionID: - return "SessionID"; - case OpNum::SetACL: - return "SetACL"; - case OpNum::GetACL: - return "GetACL"; - case OpNum::FilteredList: - return "FilteredList"; - case OpNum::CheckNotExists: - return "CheckNotExists"; - } - int32_t raw_op = static_cast(op_num); - throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); -} - OpNum getOpNum(int32_t raw_op_num) { if (!VALID_OPERATIONS.contains(raw_op_num)) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 6b50c5c5d09..a773fbbab74 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -31,6 +31,7 @@ enum class OpNum : int32_t List = 12, Check = 13, Multi = 14, + Reconfig = 16, MultiRead = 22, Auth = 100, @@ -41,7 +42,6 @@ enum class OpNum : int32_t SessionID = 997, /// Special internal request }; -std::string toString(OpNum op_num); OpNum getOpNum(int32_t raw_op_num); static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 0f27d078234..5e16a437be3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -35,6 +35,7 @@ namespace ProfileEvents extern const Event ZooKeeperRemove; extern const Event ZooKeeperExists; extern const Event ZooKeeperMulti; + extern const Event ZooKeeperReconfig; extern const Event ZooKeeperGet; extern const Event ZooKeeperSet; extern const Event ZooKeeperList; @@ -571,7 +572,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) if (err != Error::ZOK) throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}", - static_cast(err), errorMessage(err)); + static_cast(err), err); } void ZooKeeper::sendThread() @@ -697,7 +698,7 @@ void ZooKeeper::receiveThread() if (earliest_operation) { throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response in {} ms) for request {} for path: {}", - args.operation_timeout_ms, toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath()); + args.operation_timeout_ms, earliest_operation->request->getOpNum(), earliest_operation->request->getPath()); } waited_us += max_wait_us; if (waited_us >= args.session_timeout_ms * 1000) @@ -738,7 +739,7 @@ void ZooKeeper::receiveEvent() if (xid == PING_XID) { if (err != Error::ZOK) - throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", errorMessage(err)); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", err); response = std::make_shared(); } @@ -1195,7 +1196,6 @@ void ZooKeeper::create( ProfileEvents::increment(ProfileEvents::ZooKeeperCreate); } - void ZooKeeper::remove( const String & path, int32_t version, @@ -1335,6 +1335,26 @@ void ZooKeeper::sync( ProfileEvents::increment(ProfileEvents::ZooKeeperSync); } +void ZooKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + ZooKeeperReconfigRequest request; + request.joining = joining; + request.leaving = leaving; + request.new_members = new_members; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperReconfig); +} void ZooKeeper::multi( const Requests & requests, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 44ea993947e..7e27608d0a1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -178,6 +178,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 84cbb0ab7c5..675001d51e0 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -1,5 +1,4 @@ #pragma once - #include namespace DB @@ -14,8 +13,8 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -const std::string keeper_system_path = "/keeper"; -const std::string keeper_api_version_path = keeper_system_path + "/api_version"; -const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; - +const String keeper_system_path = "/keeper"; +const String keeper_api_version_path = keeper_system_path + "/api_version"; +const String keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; +const String keeper_config_path = keeper_system_path + "/config"; } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 229dbd51ab2..4fb552f20a3 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,8 @@ #pragma once - -#include - #include -#include #include +#include +#include #include #include @@ -12,6 +10,8 @@ namespace DB { +class KeeperDispatcher; + class KeeperContext { public: @@ -51,6 +51,7 @@ public: const KeeperFeatureFlags & getFeatureFlags() const; void dumpConfiguration(WriteBufferFromOwnString & buf) const; + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -85,6 +86,7 @@ private: std::unordered_map system_nodes_with_data; KeeperFeatureFlags feature_flags; + KeeperDispatcher * dispatcher{nullptr}; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..178453b2f5b 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -38,6 +38,8 @@ namespace ProfileEvents extern const Event MemoryAllocatorPurgeTimeMicroseconds; } +using namespace std::chrono_literals; + namespace DB { @@ -336,6 +338,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf keeper_context = std::make_shared(standalone_keeper); keeper_context->initialize(config); + keeper_context->dispatcher = this; server = std::make_unique( configuration_and_settings, @@ -392,7 +395,10 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf /// Start it after keeper server start session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); - update_configuration_thread = ThreadFromGlobalPool([this] { updateConfigurationThread(); }); + + update_configuration_thread = reconfigEnabled() + ? ThreadFromGlobalPool([this] { clusterUpdateThread(); }) + : ThreadFromGlobalPool([this] { clusterUpdateWithReconfigDisabledThread(); }); LOG_DEBUG(log, "Dispatcher initialized"); } @@ -429,7 +435,7 @@ void KeeperDispatcher::shutdown() if (snapshot_thread.joinable()) snapshot_thread.join(); - update_configuration_queue.finish(); + cluster_update_queue.finish(); if (update_configuration_thread.joinable()) update_configuration_thread.join(); } @@ -608,7 +614,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession "Could not push error response xid {} zxid {} error message {} to responses queue", response->xid, response->zxid, - errorMessage(error)); + error); } } @@ -653,7 +659,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { if (response->getOpNum() != Coordination::OpNum::SessionID) promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect response of type {} instead of SessionID response", Coordination::toString(response->getOpNum())))); + "Incorrect response of type {} instead of SessionID response", response->getOpNum()))); auto session_id_response = dynamic_cast(*response); if (session_id_response.internal_id != internal_id) @@ -685,17 +691,12 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) return future.get(); } - -void KeeperDispatcher::updateConfigurationThread() +void KeeperDispatcher::clusterUpdateWithReconfigDisabledThread() { - while (true) + while (!shutdown_called) { - if (shutdown_called) - return; - try { - using namespace std::chrono_literals; if (!server->checkInit()) { LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished"); @@ -710,11 +711,10 @@ void KeeperDispatcher::updateConfigurationThread() continue; } - ConfigUpdateAction action; - if (!update_configuration_queue.pop(action)) + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) break; - /// We must wait this update from leader or apply it ourself (if we are leader) bool done = false; while (!done) @@ -727,15 +727,13 @@ void KeeperDispatcher::updateConfigurationThread() if (isLeader()) { - server->applyConfigurationUpdate(action); + server->applyConfigUpdateWithReconfigDisabled(action); done = true; } - else - { - done = server->waitConfigurationUpdate(action); - if (!done) - LOG_INFO(log, "Cannot wait for configuration update, maybe we become leader, or maybe update is invalid, will try to wait one more time"); - } + else if (done = server->waitForConfigUpdateWithReconfigDisabled(action); !done) + LOG_INFO(log, + "Cannot wait for configuration update, maybe we became leader " + "or maybe update is invalid, will try to wait one more time"); } } catch (...) @@ -745,6 +743,46 @@ void KeeperDispatcher::updateConfigurationThread() } } +void KeeperDispatcher::clusterUpdateThread() +{ + while (!shutdown_called) + { + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) + return; + + if (server->applyConfigUpdate(action)) + LOG_DEBUG(log, "Processing config update {}: accepted", action); + else // TODO (myrrc) sleep a random amount? sleep less? + { + (void)cluster_update_queue.pushFront(action); + LOG_DEBUG(log, "Processing config update {}: declined, backoff", action); + std::this_thread::sleep_for(50ms); + } + } +} + +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +{ + if (shutdown_called) return; + for (auto && action : actions) + { + if (!cluster_update_queue.push(std::move(action))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push configuration update"); + LOG_DEBUG(log, "Processing config update {}: pushed", action); + } +} + +bool KeeperDispatcher::clusterUpdateQueueEmpty() const +{ + return cluster_update_queue.empty(); +} + +bool KeeperDispatcher::reconfigEnabled() const +{ + return server->reconfigEnabled(); +} + bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); @@ -752,20 +790,25 @@ bool KeeperDispatcher::isServerActive() const void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { - auto diff = server->getConfigurationDiff(config); + auto diff = server->getRaftConfigurationDiff(config); + if (diff.empty()) - LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); + LOG_TRACE(log, "Configuration update triggered, but nothing changed for Raft"); + else if (reconfigEnabled()) + LOG_WARNING(log, + "Raft configuration changed, but keeper_server.enable_reconfiguration is on. " + "This update will be ignored. Use \"reconfig\" instead"); else if (diff.size() > 1) - LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); + LOG_WARNING(log, + "Configuration changed for more than one server ({}) from cluster, " + "it's strictly not recommended", diff.size()); else LOG_DEBUG(log, "Configuration change size ({})", diff.size()); - for (auto & change : diff) - { - bool push_result = update_configuration_queue.push(change); - if (!push_result) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); - } + if (!reconfigEnabled()) + for (auto & change : diff) + if (!cluster_update_queue.push(change)) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); snapshot_s3.updateS3Configuration(config, macros); } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 1b44f0f6ced..a9b3d33eb51 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -31,7 +31,7 @@ private: using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; - using UpdateConfigurationQueue = ConcurrentBoundedQueue; + using ClusterUpdateQueue = ConcurrentBoundedQueue; /// Size depends on coordination settings std::unique_ptr requests_queue; @@ -39,7 +39,7 @@ private: SnapshotsQueue snapshots_queue{1}; /// More than 1k updates is definitely misconfiguration. - UpdateConfigurationQueue update_configuration_queue{1000}; + ClusterUpdateQueue cluster_update_queue{1000}; std::atomic shutdown_called{false}; @@ -91,8 +91,10 @@ private: void sessionCleanerTask(); /// Thread create snapshots in the background void snapshotThread(); - /// Thread apply or wait configuration changes from leader - void updateConfigurationThread(); + + // TODO (myrrc) this should be removed once "reconfig" is stabilized + void clusterUpdateWithReconfigDisabledThread(); + void clusterUpdateThread(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); @@ -132,10 +134,10 @@ public: /// and achieved quorum bool isServerActive() const; - /// Registered in ConfigReloader callback. Add new configuration changes to - /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. - /// 'macros' are used to substitute macros in endpoint of disks void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); + void pushClusterUpdates(ClusterUpdateActions&& actions); + bool clusterUpdateQueueEmpty() const; + bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp new file mode 100644 index 00000000000..dec3e1f155f --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -0,0 +1,92 @@ +#include "KeeperReconfiguration.h" +#include +#include +#include +#include + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) +{ + ClusterUpdateActions out; + std::unordered_set endpoints; + + for (const auto & server : cfg->get_servers()) + endpoints.emplace(server->get_endpoint()); + + // We can either add new servers or change weight of existing ones. + // It makes no sense having a server in _joining_ which is identical to existing one including + // weight, so such requests are declined. + for (const RaftServerConfig & update : parseRaftServers(joining)) + if (auto server_ptr = cfg->get_server(update.id)) + { + if (update.endpoint != server_ptr->get_endpoint() || update.learner != server_ptr->is_learner() + || update.priority == server_ptr->get_priority()) + return {}; // can't change server endpoint/type due to NuRaft API limitations + out.emplace_back(UpdateRaftServerPriority{.id = update.id, .priority = update.priority}); + } + else if (endpoints.contains(update.endpoint)) + return {}; + else + out.emplace_back(AddRaftServer{update}); + + return out; +} + +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving) +{ + std::vector leaving_arr; + splitInto<','>(leaving_arr, leaving); + if (leaving_arr.size() >= cfg->get_servers().size()) + return {}; + + std::unordered_set remove_ids; + ClusterUpdateActions out; + + for (std::string_view leaving_server : leaving_arr) + { + int id; + if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + return {}; + + if (remove_ids.contains(id)) + continue; + + if (auto ptr = cfg->get_server(id)) + out.emplace_back(RemoveRaftServer{.id = id}); + else + return {}; + + remove_ids.emplace(id); + } + + return out; +} + +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates) +{ + RaftServers new_config; + std::unordered_set remove_update_ids; + + for (const auto & update : updates) + { + if (const auto * add = std::get_if(&update)) + new_config.emplace_back(*add); + else if (const auto * remove = std::get_if(&update)) + remove_update_ids.insert(remove->id); + else if (const auto * priority = std::get_if(&update)) + { + remove_update_ids.insert(priority->id); + new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); + } + else + UNREACHABLE(); + } + + for (const auto & item : cfg->get_servers()) + if (!remove_update_ids.contains(item->get_id())) + new_config.emplace_back(RaftServerConfig{*item}); + + return fmt::format("{}", fmt::join(new_config.begin(), new_config.end(), "\n")); +} +} diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h new file mode 100644 index 00000000000..71958f2035e --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.h @@ -0,0 +1,10 @@ +#pragma once +#include "Coordination/KeeperSnapshotManager.h" +#include "Coordination/RaftServerConfig.h" + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining); +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving); +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates = {}); +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 82c843287c1..1cde957ef3a 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB { @@ -40,6 +41,8 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } +using namespace std::chrono_literals; + namespace { @@ -118,6 +121,7 @@ KeeperServer::KeeperServer( , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) + , enable_reconfiguration(config.getBool("keeper_server.enable_reconfiguration", false)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -450,7 +454,7 @@ void KeeperServer::shutdownRaftServer() size_t count = 0; while (asio_service->get_active_workers() != 0 && count < timeout * 100) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); + std::this_thread::sleep_for(10ms); count++; } } @@ -715,10 +719,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (next_index < last_commited || next_index - last_commited <= 1) commited_store = true; - auto set_initialized = [this]() + auto set_initialized = [this] { - std::lock_guard lock(initialized_mutex); - initialized_flag = true; + { + std::lock_guard lock(initialized_mutex); + initialized_flag = true; + } initialized_cv.notify_all(); }; @@ -783,9 +789,42 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { - auto diff = state_manager->getConfigurationDiff(config); + std::lock_guard _{server_write_mutex}; + + if (const auto* add = std::get_if(&action)) + return raft_instance->get_srv_config(add->id) != nullptr + || raft_instance->add_srv(static_cast(*add))->get_accepted(); + else if (const auto* remove = std::get_if(&action)) + { + if (isLeader() && remove->id == state_manager->server_id()) + { + raft_instance->yield_leadership(); + return false; + } + + return raft_instance->get_srv_config(remove->id) == nullptr + || raft_instance->remove_srv(remove->id)->get_accepted(); + } + else if (const auto* update = std::get_if(&action)) + { + if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) + throw Exception(ErrorCodes::RAFT_ERROR, + "Attempt to apply {} but server is not present in Raft", + action); + else if (ptr->get_priority() == update->priority) + return true; + + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return true; + } + UNREACHABLE(); +} + +ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +{ + auto diff = state_manager->getRaftConfigurationDiff(config); if (!diff.empty()) { @@ -796,160 +835,103 @@ ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::Abstrac return diff; } -void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) +void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - std::lock_guard lock{server_write_mutex}; - if (is_recovering) - return; + std::lock_guard _{server_write_mutex}; + if (is_recovering) return; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to apply {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); }; + auto not_leader = [&] { LOG_INFO(log, "Not leader anymore, aborting"); }; + auto backoff_on_refusal = [&](size_t i) + { + LOG_INFO(log, "Update was not accepted (try {}), backing off for {}", i + 1, sleep_time * (i + 1)); + std::this_thread::sleep_for(sleep_time * (i + 1)); + }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to add server with id {}", task.server->get_id()); - bool added = false; for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); - added = true; - break; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to add server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->add_srv(*task.server); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->add_srv(static_cast(*add))->get_accepted()) + backoff_on_refusal(i); } - if (!added) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to add server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); - - bool removed = false; - if (task.server->get_id() == state_manager->server_id()) + if (remove->id == state_manager->server_id()) { - LOG_INFO( - log, - "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. " + LOG_INFO(log, + "Trying to remove leader node (ourself), so will yield leadership and some other node " + "(new leader) will try to remove us. " "Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node"); - - raft_instance->yield_leadership(); - return; + return raft_instance->yield_leadership(); } for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); - removed = true; - break; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to remove server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->remove_srv(task.server->get_id()); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->remove_srv(remove->id)->get_accepted()) + backoff_on_refusal(i); } - if (!removed) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) - raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + else if (const auto* update = std::get_if(&action)) + { + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return; + } + + throw Exception(ErrorCodes::RAFT_ERROR, + "Configuration change {} was not accepted by Raft after {} retries", + action, coordination_settings->configuration_change_tries_count); } - -bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) +bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - if (is_recovering) - return false; + if (is_recovering) return false; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to wait for {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); return true; }; + auto became_leader = [&] { LOG_INFO(log, "Became leader, aborting"); return false; }; + auto backoff = [&](size_t i) { std::this_thread::sleep_for(sleep_time * (i + 1)); }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id()); for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to add server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id()); - for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to remove server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + else if (std::get_if(&action) != nullptr) return true; - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); - return true; + + return false; } Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 8f416b1f48c..61e29b67bbd 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -10,12 +10,15 @@ #include #include #include +#include namespace DB { using RaftAppendResult = nuraft::ptr>>; +class KeeperDispatcher; + class KeeperServer { private: @@ -28,9 +31,10 @@ private: nuraft::ptr state_manager; struct KeeperRaftServer; - nuraft::ptr raft_instance; + nuraft::ptr raft_instance; // TSA_GUARDED_BY(server_write_mutex); nuraft::ptr asio_service; std::vector> asio_listeners; + // because some actions can be applied // when we are sure that there are no requests currently being // processed (e.g. recovery) we do all write actions @@ -65,6 +69,7 @@ private: std::shared_ptr keeper_context; const bool create_snapshot_on_exit; + const bool enable_reconfiguration; public: KeeperServer( @@ -84,6 +89,7 @@ public: void putLocalReadRequest(const KeeperStorage::RequestForSession & request); bool isRecovering() const { return is_recovering; } + bool reconfigEnabled() const { return enable_reconfiguration; } /// Put batch of requests into Raft and get result of put. Responses will be set separately into /// responses_queue. @@ -122,17 +128,12 @@ public: int getServerID() const { return server_id; } - /// Get configuration diff between current configuration in RAFT and in XML file - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); + bool applyConfigUpdate(const ClusterUpdateAction& action); - /// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. - /// Synchronously check for update results with retries. - void applyConfigurationUpdate(const ConfigUpdateAction & task); - - - /// Wait configuration update for action. Used by followers. - /// Return true if update was successfully received. - bool waitConfigurationUpdate(const ConfigUpdateAction & task); + // TODO (myrrc) these functions should be removed once "reconfig" is stabilized + void applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + bool waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config); uint64_t createSnapshot(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..3e9850caa40 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -2,17 +2,20 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include #include #include #include "Coordination/KeeperStorage.h" +#include "Coordination/KeeperReconfiguration.h" #include @@ -146,7 +149,7 @@ void assertDigest( "Digest for nodes is not matching after {} request of type '{}'.\nExpected digest - {}, actual digest - {} (digest " "{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}", committing ? "committing" : "preprocessing", - Coordination::toString(request.getOpNum()), + request.getOpNum(), first.value, second.value, first.version, @@ -261,7 +264,8 @@ std::shared_ptr KeeperStateMachine::parseReque bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) { - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + const auto op_num = request_for_session.request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; std::lock_guard lock(storage_and_responses_lock); @@ -291,14 +295,89 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) +{ + const auto& request = static_cast(*request_for_session.request); + const int64_t session_id = request_for_session.session_id; + const int64_t zxid = request_for_session.zxid; + + using enum Coordination::Error; + auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession + { + auto res = std::make_shared(); + res->xid = request.xid; + res->zxid = zxid; + res->error = code; + return { session_id, std::move(res) }; + }; + + KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + if (!dispatcher.reconfigEnabled()) + return bad_request(ZUNIMPLEMENTED); + if (!dispatcher.clusterUpdateQueueEmpty()) + return bad_request(ZRECONFIGINPROGRESS); + if (request.version != -1) + return bad_request(ZBADVERSION); + + const bool has_new_members = !request.new_members.empty(); + const bool has_joining = !request.joining.empty(); + const bool has_leaving = !request.leaving.empty(); + const bool incremental_reconfig = (has_joining || has_leaving) && !has_new_members; + if (!incremental_reconfig) + return bad_request(); + + const ClusterConfigPtr config = getClusterConfig(); + if (!config) // Server can be uninitialized yet + return bad_request(); + + ClusterUpdateActions updates; + + if (has_joining) + { + if (auto join_updates = joiningToClusterUpdates(config, request.joining); !join_updates.empty()) + moveExtend(updates, std::move(join_updates)); + else + return bad_request(); + } + + if (has_leaving) + { + if (auto leave_updates = leavingToClusterUpdates(config, request.leaving); !leave_updates.empty()) + moveExtend(updates, std::move(leave_updates)); + else + return bad_request(); + } + + auto response = std::make_shared(); + response->xid = request.xid; + response->zxid = zxid; + response->error = Coordination::Error::ZOK; + response->value = serializeClusterConfig(config, updates); + + dispatcher.pushClusterUpdates(std::move(updates)); + return { session_id, std::move(response) }; +} + nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, true); if (!request_for_session->zxid) request_for_session->zxid = log_idx; - /// Special processing of session_id request - if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID) + auto try_push = [this](const KeeperStorage::ResponseForSession& response) + { + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } + }; + + const auto op_num = request_for_session->request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDRequest & session_id_request = dynamic_cast(*request_for_session->request); @@ -309,21 +388,24 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponseForSession response_for_session; response_for_session.session_id = -1; response_for_session.response = response; - { - std::lock_guard lock(storage_and_responses_lock); - session_id = storage->getSessionID(session_id_request.session_timeout_ms); - LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); - response->session_id = session_id; - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", session_id); - } - } + + std::lock_guard lock(storage_and_responses_lock); + session_id = storage->getSessionID(session_id_request.session_timeout_ms); + LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); + response->session_id = session_id; + try_push(response_for_session); + } + // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies + // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. + // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details + else if (op_num == Coordination::OpNum::Reconfig) + { + std::lock_guard lock(storage_and_responses_lock); + try_push(processReconfiguration(*request_for_session)); } else { - if (request_for_session->request->getOpNum() == Coordination::OpNum::Close) + if (op_num == Coordination::OpNum::Close) { std::lock_guard lock(request_cache_mutex); parsed_request_cache.erase(request_for_session->session_id); @@ -333,14 +415,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING( - log, - "Failed to push response with session id {} to the queue, probably because of shutdown", - response_for_session.session_id); - } + try_push(response_for_session); if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); @@ -782,5 +857,4 @@ void KeeperStateMachine::recalculateStorageStats() storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); } - } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index b47a9b5cc42..3b239adae45 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,6 +12,7 @@ namespace DB { +class KeeperDispatcher; using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; @@ -67,7 +68,9 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); - void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock( + const KeeperStorage::RequestForSession & request_for_session, + bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; uint64_t last_commit_index() override { return last_committed_idx; } @@ -87,8 +90,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - /// just for test - KeeperStorage & getStorage() { return *storage; } + KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + { + return *storage; + } void shutdownStorage(); @@ -122,6 +127,7 @@ public: uint64_t getLatestSnapshotBufSize() const; void recalculateStorageStats(); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored @@ -133,7 +139,7 @@ private: CoordinationSettingsPtr coordination_settings; /// Main state machine logic - KeeperStoragePtr storage; + KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; @@ -178,6 +184,8 @@ private: KeeperContextPtr keeper_context; KeeperSnapshotManagerS3 * snapshot_manager_s3; + + KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); }; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 450fd04b61d..cf1bad8c5fa 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -451,7 +451,7 @@ nuraft::ptr KeeperStateManager::read_state() return nullptr; } -ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const +ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const { auto new_configuration_wrapper = parseServersConfiguration(config, true); @@ -465,14 +465,14 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A old_ids[old_server->get_id()] = old_server; } - ConfigUpdateActions result; + ClusterUpdateActions result; /// First of all add new servers for (const auto & [new_id, server_config] : new_ids) { auto old_server_it = old_ids.find(new_id); if (old_server_it == old_ids.end()) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + result.emplace_back(AddRaftServer{RaftServerConfig{*server_config}}); else { const auto & old_endpoint = old_server_it->second->get_endpoint(); @@ -491,10 +491,8 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A /// After that remove old ones for (auto [old_id, server_config] : old_ids) - { if (!new_ids.contains(old_id)) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); - } + result.emplace_back(RemoveRaftServer{old_id}); { std::lock_guard lock(configuration_wrapper_mutex); @@ -507,7 +505,10 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A { if (old_server->get_priority() != new_server->get_priority()) { - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + result.emplace_back(UpdateRaftServerPriority{ + .id = new_server->get_id(), + .priority = new_server->get_priority() + }); } break; } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index f24f0c2b1e5..5abeea604b5 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -7,31 +7,13 @@ #include #include #include "Coordination/KeeperStateMachine.h" +#include "Coordination/RaftServerConfig.h" #include namespace DB { - using KeeperServerConfigPtr = nuraft::ptr; -/// When our configuration changes the following action types -/// can happen -enum class ConfigUpdateActionType -{ - RemoveServer, - AddServer, - UpdatePriority, -}; - -/// Action to update configuration -struct ConfigUpdateAction -{ - ConfigUpdateActionType action_type; - KeeperServerConfigPtr server; -}; - -using ConfigUpdateActions = std::vector; - /// Responsible for managing our and cluster configuration class KeeperStateManager : public nuraft::state_mgr { @@ -74,7 +56,11 @@ public: int32_t server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return configuration_wrapper.config; } /// NOLINT + nuraft::ptr get_srv_config() const + { + std::lock_guard lk(configuration_wrapper_mutex); + return configuration_wrapper.config; + } void system_exit(const int exit_code) override; /// NOLINT @@ -106,8 +92,8 @@ public: /// Read all log entries in log store from the begging and return latest config (with largest log_index) ClusterConfigPtr getLatestConfigFromLogStore() const; - /// Get configuration diff between proposed XML and current state in RAFT - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; + // TODO (myrrc) This should be removed once "reconfig" is stabilized + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: const String & getOldServerStatePath(); @@ -133,7 +119,7 @@ private: std::string config_prefix; mutable std::mutex configuration_wrapper_mutex; - KeeperConfigurationWrapper configuration_wrapper; + KeeperConfigurationWrapper configuration_wrapper TSA_GUARDED_BY(configuration_wrapper_mutex); nuraft::ptr log_store; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 884aacc4558..2b245a455b7 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -20,10 +20,10 @@ #include #include +#include #include +#include -#include -#include #include #include #include @@ -53,7 +53,6 @@ namespace ErrorCodes namespace { - String getSHA1(const String & userdata) { Poco::SHA1Engine engine; @@ -1060,7 +1059,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_feature_flags_path) + if (request.path == Coordination::keeper_api_feature_flags_path + || request.path == Coordination::keeper_config_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) @@ -1085,6 +1085,14 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } } + if (request.path == Coordination::keeper_config_path) + { + response.data = serializeClusterConfig( + storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + auto & container = storage.container; auto node_it = container.find(request.path); if (node_it == container.end()) @@ -1784,7 +1792,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", - Coordination::toString(sub_zk_request->getOpNum())); + sub_zk_request->getOpNum()); } } @@ -1975,7 +1983,7 @@ public: { auto request_it = op_num_to_request.find(zk_request->getOpNum()); if (request_it == op_num_to_request.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", toString(zk_request->getOpNum())); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum()); return request_it->second(zk_request); } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp new file mode 100644 index 00000000000..42923dd0b29 --- /dev/null +++ b/src/Coordination/RaftServerConfig.cpp @@ -0,0 +1,96 @@ +#include "RaftServerConfig.h" +#include +#include +#include +#include + +namespace DB +{ +RaftServerConfig::RaftServerConfig(const nuraft::srv_config & cfg) noexcept + : id(cfg.get_id()), endpoint(cfg.get_endpoint()), learner(cfg.is_learner()), priority(cfg.get_priority()) +{ +} + +RaftServerConfig::operator nuraft::srv_config() const noexcept +{ + return {id, 0, endpoint, "", learner, priority}; +} + +std::optional RaftServerConfig::parse(std::string_view server) noexcept +{ + std::vector parts; + splitInto<';', '='>(parts, server); + + const bool with_id_endpoint = parts.size() == 2; + const bool with_server_type = parts.size() == 3; + const bool with_priority = parts.size() == 4; + if (!with_id_endpoint && !with_server_type && !with_priority) + return std::nullopt; + + const std::string_view id_str = parts[0]; + if (!id_str.starts_with("server.")) + return std::nullopt; + + int id; + if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + return std::nullopt; + if (id <= 0) + return std::nullopt; + + const std::string_view endpoint = parts[1]; + const size_t port_delimiter = endpoint.find_last_of(':'); + if (port_delimiter == std::string::npos) + return {}; + const std::string_view port = endpoint.substr(port_delimiter + 1); + + uint16_t port_tmp; + if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + return std::nullopt; + + RaftServerConfig out{id, endpoint}; + + if (with_id_endpoint) + return out; + + if (parts[2] != "learner" && parts[2] != "participant") + return std::nullopt; + out.learner = parts[2] == "learner"; + if (with_server_type) + return out; + + const std::string_view priority = parts[3]; + if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + return std::nullopt; + if (out.priority < 0) + return std::nullopt; + + return out; +} + +RaftServers parseRaftServers(std::string_view servers) +{ + std::vector server_arr; + std::unordered_set ids; + std::unordered_set endpoints; + RaftServers out; + + for (auto & server : splitInto<','>(server_arr, servers)) + if (auto maybe_server = RaftServerConfig::parse(server)) + { + String endpoint = maybe_server->endpoint; + if (endpoints.contains(endpoint)) + return {}; + const int id = maybe_server->id; + if (ids.contains(id)) + return {}; + + out.emplace_back(std::move(*maybe_server)); + endpoints.emplace(std::move(endpoint)); + ids.emplace(id); + } + else + return {}; + + return out; +} +} diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h new file mode 100644 index 00000000000..451d61a436e --- /dev/null +++ b/src/Coordination/RaftServerConfig.h @@ -0,0 +1,78 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ +// default- and copy-constructible version of nuraft::srv_config +struct RaftServerConfig +{ + int id; + String endpoint; + bool learner; + int priority; + + constexpr RaftServerConfig() = default; + constexpr RaftServerConfig(int id_, std::string_view endpoint_, bool learner_ = false, int priority_ = 1) + : id(id_), endpoint(endpoint_), learner(learner_), priority(priority_) + { + } + + constexpr bool operator==(const RaftServerConfig &) const = default; + explicit RaftServerConfig(const nuraft::srv_config & cfg) noexcept; + explicit operator nuraft::srv_config() const noexcept; + + /// Parse server in format "server.id=host:port[;learner][;priority]" + static std::optional parse(std::string_view server) noexcept; +}; + +using RaftServers = std::vector; +/// Parse comma-delimited servers. Check for duplicate endpoints and ids. +/// @returns {} on parsing or validation error. +RaftServers parseRaftServers(std::string_view servers); + +struct AddRaftServer : RaftServerConfig +{ +}; + +struct RemoveRaftServer +{ + int id; +}; + +struct UpdateRaftServerPriority +{ + int id; + int priority; +}; + +using ClusterUpdateAction = std::variant; +using ClusterUpdateActions = std::vector; +} + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + { + return fmt::format_to( + ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); + } +}; + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + { + if (const auto * add = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Add server {})", add->id); + if (const auto * remove = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Remove server {})", remove->id); + if (const auto * update = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Change server {} priority to {})", update->id, update->priority); + UNREACHABLE(); + } +}; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 0f60c960b8b..b302f9b13ca 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -84,6 +84,47 @@ protected: } }; +TEST_P(CoordinationTest, RaftServerConfigParse) +{ + auto parse = Coordination::RaftServerConfig::parse; + using Cfg = std::optional; + + EXPECT_EQ(parse(""), std::nullopt); + EXPECT_EQ(parse("="), std::nullopt); + EXPECT_EQ(parse("=;"), std::nullopt); + EXPECT_EQ(parse("=;;"), std::nullopt); + EXPECT_EQ(parse("=:80"), std::nullopt); + EXPECT_EQ(parse("server."), std::nullopt); + EXPECT_EQ(parse("server.=:80"), std::nullopt); + EXPECT_EQ(parse("server.-5=1:2"), std::nullopt); + EXPECT_EQ(parse("server.1=host;-123"), std::nullopt); + EXPECT_EQ(parse("server.1=host:999"), (Cfg{{1, "host:999"}})); + EXPECT_EQ(parse("server.1=host:999;learner"), (Cfg{{1, "host:999", true}})); + EXPECT_EQ(parse("server.1=host:999;participant"), (Cfg{{1, "host:999", false}})); + EXPECT_EQ(parse("server.1=host:999;learner;25"), (Cfg{{1, "host:999", true, 25}})); + + EXPECT_EQ(parse("server.1=127.0.0.1:80"), (Cfg{{1, "127.0.0.1:80"}})); + EXPECT_EQ( + parse("server.1=2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"), + (Cfg{{1, "2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"}})); +} + +TEST_P(CoordinationTest, RaftServerClusterConfigParse) +{ + auto parse = Coordination::parseRaftServers; + using Cfg = DB::RaftServerConfig; + using Servers = DB::RaftServers; + + EXPECT_EQ(parse(""), Servers{}); + EXPECT_EQ(parse(","), Servers{}); + EXPECT_EQ(parse("1,2"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.1=host2:80"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.2=host:80"), Servers{}); + EXPECT_EQ( + parse("server.1=host:80,server.2=host:81"), + (Servers{Cfg{1, "host:80"}, Cfg{2, "host:81"}})); +} + TEST_P(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; @@ -1575,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorage(); - auto & restored_storage = restore_machine->getStorage(); + auto & source_storage = state_machine->getStorageForUnitTests(); + auto & restored_storage = restore_machine->getStorageForUnitTests(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1678,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorage(); + const auto & storage = state_machine->getStorageForUnitTests(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1727,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1790,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 48f4d510af7..2231a58c6a9 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -73,6 +73,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"Create", static_cast(Coordination::OpNum::Create)}, {"Remove", static_cast(Coordination::OpNum::Remove)}, {"Exists", static_cast(Coordination::OpNum::Exists)}, + {"Reconfig", static_cast(Coordination::OpNum::Reconfig)}, {"Get", static_cast(Coordination::OpNum::Get)}, {"Set", static_cast(Coordination::OpNum::Set)}, {"GetACL", static_cast(Coordination::OpNum::GetACL)}, diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp index a1f35a5ae42..78d69c83989 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/DataLakes/HudiMetadataParser.cpp @@ -67,7 +67,8 @@ struct HudiMetadataParser::Impl { auto key_file = std::filesystem::path(key); Strings file_parts; - splitInto<'_'>(file_parts, key_file.stem()); + const String stem = key_file.stem(); + splitInto<'_'>(file_parts, stem); if (file_parts.size() != 3) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index bcc4dc749fb..07cfced8362 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -492,7 +492,7 @@ size_t ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_di } else { - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, Coordination::errorMessage(rc)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, rc); } first_outdated_block++; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 792843cbe18..b08b9de12a3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -494,7 +494,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't set value of nodes for insert times " "({}/min_unprocessed_insert_time, max_processed_insert_time): {}. " - "This shouldn't happen often.", replica_path, Coordination::errorMessage(code)); + "This shouldn't happen often.", replica_path, code); } } @@ -551,7 +551,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); + LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, code); updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); } @@ -1144,7 +1144,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / znode_name); if (code != Coordination::Error::ZOK) - LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), Coordination::errorMessage(code)); + LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), code); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4128654a632..1e033566fed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -723,7 +723,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Insert failed due to zookeeper error. Please retry. Reason: {}", - Coordination::errorMessage(write_part_info_keeper_error)); + write_part_info_keeper_error); } retries_ctl.stopRetries(); @@ -1033,7 +1033,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNKNOWN_STATUS_OF_INSERT, "Unknown status, client must retry. Reason: {}", - Coordination::errorMessage(multi_code)); + multi_code); return; } else if (Coordination::isUserError(multi_code)) @@ -1109,7 +1109,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected logical error while adding block {} with ID '{}': {}, path {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code), + multi_code, failed_op_path); } } @@ -1122,7 +1122,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code)); + multi_code); } }, [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dac9e6923a5..06e9d88a954 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1037,7 +1037,7 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (replica: {}). Will remove recursively.", - Coordination::errorMessage(code), remote_replica_path); + code, remote_replica_path); /// And finally remove everything else recursively /// It may left some garbage if replica_path subtree is concurrently modified @@ -1145,7 +1145,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper auto code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (table: {}). Will remove recursively.", - Coordination::errorMessage(code), zookeeper_path); + code, zookeeper_path); Strings children; code = zookeeper->tryGetChildren(zookeeper_path, children); @@ -1893,7 +1893,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", - entry.new_part_name, Coordination::errorMessage(code)); + entry.new_part_name, code); } else throw Coordination::Exception(code); @@ -3098,7 +3098,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_is_lost.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, Coordination::errorMessage(get_is_lost.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, get_is_lost.error); continue; } else if (get_is_lost.data != "0") @@ -3109,12 +3109,12 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_log_pointer.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, Coordination::errorMessage(get_log_pointer.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, get_log_pointer.error); continue; } if (get_queue.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, Coordination::errorMessage(get_queue.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, get_queue.error); continue; } @@ -7203,7 +7203,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( { for (size_t i = 0; i < delete_requests.size(); ++i) if (delete_responses[i]->error != Coordination::Error::ZOK) - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), Coordination::errorMessage(delete_responses[i]->error)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), delete_responses[i]->error); } LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", delete_requests.size(), partition_id); @@ -8717,7 +8717,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { - LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), code); return {true, std::nullopt}; } diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3b909194b63..3da1d5bd7b0 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,5 +1,6 @@ import socket import time +from kazoo.client import KazooClient def get_keeper_socket(cluster, node, port=9181): @@ -26,9 +27,17 @@ def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" -def wait_until_connected(cluster, node, port=9181): +def wait_until_connected(cluster, node, port=9181, timeout=30.0): + elapsed = 0.0 + while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG: time.sleep(0.1) + elapsed += 0.1 + + if elapsed >= timeout: + raise Exception( + f"{timeout}s timeout while waiting for {node.name} to start serving requests" + ) def wait_until_quorum_lost(cluster, node, port=9181): @@ -51,3 +60,25 @@ def get_leader(cluster, nodes): if is_leader(cluster, node): return node raise Exception("No leader in Keeper cluster.") + + +def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: + _fake = KazooClient( + hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout + ) + _fake.start() + return _fake + + +def get_config_str(zk: KazooClient) -> str: + """ + Return decoded contents of /keeper/config node + """ + return zk.get("/keeper/config")[0].decode("utf-8") + + +def configs_equal(left: str, right: str) -> bool: + """ + Check whether /keeper/config nodes are equal + """ + return sorted(left.split("\n")) == sorted(right.split("\n")) diff --git a/tests/integration/test_keeper_nodes_move/test.py b/tests/integration/test_keeper_nodes_move/test.py index 6884ff29607..8ac7bc9b5e2 100644 --- a/tests/integration/test_keeper_nodes_move/test.py +++ b/tests/integration/test_keeper_nodes_move/test.py @@ -1,12 +1,7 @@ #!/usr/bin/env python3 - -#!/usr/bin/env python3 - import pytest from helpers.cluster import ClickHouseCluster -import random -import string import os import time from multiprocessing.dummy import Pool diff --git a/tests/integration/test_keeper_reconfig_add/__init__.py b/tests/integration/test_keeper_reconfig_add/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml new file mode 100644 index 00000000000..44e2090e9d8 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml @@ -0,0 +1,20 @@ + + + true + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml new file mode 100644 index 00000000000..e9249f7091c --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml @@ -0,0 +1,21 @@ + + + true + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml new file mode 100644 index 00000000000..a7ff1f6de28 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml @@ -0,0 +1,22 @@ + + + true + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + 3 node3 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py new file mode 100644 index 00000000000..c80279a0727 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -0,0 +1,155 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", stay_alive=True) +node3 = cluster.add_instance("node3", stay_alive=True) + +server_join_msg = "confirms it will join" +part_of_cluster = "now this node is the part of cluster" +zk1, zk2, zk3 = None, None, None + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node2.stop_clickhouse() + node2.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper2.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + node3.stop_clickhouse() + node3.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper3.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def test_reconfig_add(started_cluster): + """ + Add a node to another node. Then add another node to two. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + with pytest.raises(BadArgumentsException): + # duplicate id with different endpoint + zk1.reconfig(joining="server.1=localhost:1337", leaving=None, new_members=None) + + with pytest.raises(BadArgumentsException): + # duplicate endpoint + zk1.reconfig(joining="server.8=node1:9234", leaving=None, new_members=None) + + for i in range(100): + zk1.create(f"/test_three_{i}", b"somedata") + + node2.start_clickhouse() + config, _ = zk1.reconfig( + joining="server.2=node2:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node2) + + config = config.decode("utf-8") + print("After adding 2", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"/test_three_{i}") is not None + zk2.create(f"/test_three_{100 + i}", b"somedata") + + # Why not both? + # One node will process add_srv request, other will pull out updated config, apply + # and return true in config update thread (without calling add_srv again) + assert node1.contains_in_log(server_join_msg) or node2.contains_in_log( + server_join_msg + ) + + assert node2.contains_in_log(part_of_cluster) + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_three_0") + + for i in range(200): + assert zk1.exists(f"/test_three_{i}") is not None + + for i in range(100): + zk2.create(f"/test_four_{i}", b"somedata") + + node3.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.3=node3:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node3) + + config = config.decode("utf-8") + print("After adding 3", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + zk3 = get_fake_zk(node3) + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk3.exists(f"/test_four_{i}") is not None + zk3.create(f"/test_four_{100 + i}", b"somedata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + + for i in range(200): + assert zk1.exists(f"/test_four_{i}") is not None + assert zk2.exists(f"/test_four_{i}") is not None + + assert node3.contains_in_log(part_of_cluster) diff --git a/tests/integration/test_keeper_reconfig_remove/__init__.py b/tests/integration/test_keeper_reconfig_remove/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml new file mode 100644 index 00000000000..bbadc2741af --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml new file mode 100644 index 00000000000..0191a522a50 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml new file mode 100644 index 00000000000..345bf402336 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py new file mode 100644 index 00000000000..7f0b1ee92c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -0,0 +1,145 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3 = None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_followers_from_3(started_cluster): + """ + Remove 1 follower node from cluster of 3. + Then remove another follower from two left nodes. + Check that remaining node is in standalone mode. + """ + + zk1 = get_fake_zk(node1) + config, _ = zk1.get("/keeper/config") + config = config.decode("utf-8") + print("Initial config", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + with pytest.raises(BadVersionException): + zk1.reconfig(joining=None, leaving="1", new_members=None, from_config=20) + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining=None, leaving=None, new_members=None) + with pytest.raises(BadArgumentsException): + # bulk reconfiguration is not supported + zk1.reconfig(joining=None, leaving=None, new_members="3") + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining="1", leaving="1", new_members="3") + with pytest.raises(BadArgumentsException): + # at least one node must be left + zk1.reconfig(joining=None, leaving="1,2,3", new_members=None) + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + assert zk3.exists(f"test_two_{i}") is not None + + config, _ = zk1.reconfig(joining=None, leaving="3", new_members=None) + config = config.decode("utf-8") + print("After removing 3", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + zk2.create(f"/test_two_{100 + i}", b"otherdata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + + assert node3.contains_in_log(log_msg_removed) + + for i in range(100): + zk2.create(f"/test_two_{200 + i}", b"otherdata") + + config, _ = zk1.reconfig(joining=None, leaving="2", new_members=None) + config = config.decode("utf-8") + + print("After removing 2", config) + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(300): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert node2.contains_in_log(log_msg_removed) + assert "Mode: standalone" in zk1.command(b"stat") diff --git a/tests/integration/test_keeper_reconfig_remove_many/__init__.py b/tests/integration/test_keeper_reconfig_remove_many/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml new file mode 100644 index 00000000000..9976169624b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml @@ -0,0 +1,47 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml new file mode 100644 index 00000000000..edc43142464 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml @@ -0,0 +1,47 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml new file mode 100644 index 00000000000..8cebcbc0808 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml @@ -0,0 +1,47 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml new file mode 100644 index 00000000000..99ac7e53f30 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml @@ -0,0 +1,47 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml new file mode 100644 index 00000000000..92102ad486b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml @@ -0,0 +1,47 @@ + + + 9181 + 5 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py new file mode 100644 index 00000000000..6bf477ff9c9 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -0,0 +1,149 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient, KazooState +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +nodes = [ + cluster.add_instance(f"node{i}", main_configs=[f"configs/keeper{i}.xml"]) + for i in range(1, 6) +] +node1, node2, node3, node4, node5 = nodes + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3, zk4, zk5 = None, None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3, zk4, zk5]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_2_and_leader(started_cluster): + """ + Remove 2 followers from a cluster of 5. Remove leader from 3 nodes. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 5 + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk5)) + + for i in range(100): + assert zk4.exists(f"test_two_{i}") is not None + assert zk5.exists(f"test_two_{i}") is not None + + zk4.create(f"/test_two_{100 + i}", b"otherdata") + + zk2 = get_fake_zk(node2) + config, _ = zk2.reconfig(joining=None, leaving="4,5", new_members=None) + config = config.decode("utf-8") + + print("After removing 4,5", config) + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + assert ku.configs_equal(config, ku.get_config_str(zk1)) + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + assert zk2.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk4.stop() + zk4.close() + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + + with pytest.raises(Exception): + zk5.stop() + zk5.close() + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) + assert node4.contains_in_log(log_msg_removed) + assert node5.contains_in_log(log_msg_removed) + + assert ku.is_leader(cluster, node1) + + for i in range(100): + zk1.create(f"/test_leader_{i}", b"somedata") + + # when a leader gets a remove request, it must yield leadership + config, _ = zk1.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + print("After removing 1 (leader)", config) + + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_leader_{i}") is not None + assert zk3.exists(f"test_leader_{i}") is not None + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_leader_0") + + assert node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py new file mode 100644 index 00000000000..1b23aa056c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -0,0 +1,127 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node via two commands. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + config, _ = zk2.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + + print("After removing 1 (leader)", config) + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving=None, new_members=None + ) + config = config.decode("utf-8") + ku.wait_until_connected(cluster, node4) + + print("After adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + zk4 = get_fake_zk(node4) + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py new file mode 100644 index 00000000000..c7aed945097 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -0,0 +1,120 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader_in_one_command(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node to this cluster in a single command + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving="1", new_members=None + ) + config = config.decode("utf-8") + + print("After removing 1 and adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + ku.wait_until_connected(cluster, node4) + time.sleep(1) + + zk4 = get_fake_zk(node4) + zk4.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 5a6fd15d72c..8d685d65d1d 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorage(); + auto & storage = machine->getStorageForUnitTests(); std::queue keys; keys.push("/"); From b4f750ed66a7579580b88deaaedf15ac153785d0 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 10:11:44 +0000 Subject: [PATCH 219/871] review fixes --- base/base/find_symbols.h | 2 +- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Coordination/KeeperContext.cpp | 3 ++- src/Coordination/KeeperContext.h | 5 +++-- src/Coordination/KeeperDispatcher.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.h | 4 ++-- src/Coordination/KeeperServer.cpp | 14 +++++++------- src/Coordination/KeeperServer.h | 2 -- src/Coordination/KeeperStateMachine.cpp | 10 ++++++---- src/Coordination/KeeperStateMachine.h | 2 -- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/RaftServerConfig.cpp | 2 +- 13 files changed, 28 insertions(+), 30 deletions(-) diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index f7d24ccfc11..fda94edaa88 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline To& splitInto(To & to, std::string_view what, bool token_compress = false) +inline To & splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 20ce2a748e6..8567a53699e 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -573,7 +573,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto& ctx) + constexpr auto format(Coordination::Error code, auto & ctx) { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 3c3c0500540..0c083971f74 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,8 +32,9 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) { + dispatcher = dispatcher_; digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 4fb552f20a3..ba1a81b4423 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -24,7 +24,7 @@ public: SHUTDOWN }; - void initialize(const Poco::Util::AbstractConfiguration & config); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_); Phase getServerState() const; void setServerState(Phase server_state_); @@ -52,6 +52,8 @@ public: void dumpConfiguration(WriteBufferFromOwnString & buf) const; + constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -90,5 +92,4 @@ private: }; using KeeperContextPtr = std::shared_ptr; - } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 178453b2f5b..26be2881780 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -337,8 +337,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config); - keeper_context->dispatcher = this; + keeper_context->initialize(config, this); server = std::make_unique( configuration_and_settings, @@ -762,7 +761,7 @@ void KeeperDispatcher::clusterUpdateThread() } } -void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) { if (shutdown_called) return; for (auto && action : actions) diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index dec3e1f155f..f262a07209d 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -1,5 +1,4 @@ #include "KeeperReconfiguration.h" -#include #include #include #include @@ -45,8 +44,8 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: for (std::string_view leaving_server : leaving_arr) { - int id; - if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + int32_t id; + if (!tryParse(id, leaving_server)) return {}; if (remove_ids.contains(id)) diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h index 71958f2035e..fdd81708da2 100644 --- a/src/Coordination/KeeperReconfiguration.h +++ b/src/Coordination/KeeperReconfiguration.h @@ -1,6 +1,6 @@ #pragma once -#include "Coordination/KeeperSnapshotManager.h" -#include "Coordination/RaftServerConfig.h" +#include +#include namespace DB { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 1cde957ef3a..f6715b0da3f 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -793,10 +793,10 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { std::lock_guard _{server_write_mutex}; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) return raft_instance->get_srv_config(add->id) != nullptr || raft_instance->add_srv(static_cast(*add))->get_accepted(); - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (isLeader() && remove->id == state_manager->server_id()) { @@ -807,7 +807,7 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) return raft_instance->get_srv_config(remove->id) == nullptr || raft_instance->remove_srv(remove->id)->get_accepted(); } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) throw Exception(ErrorCodes::RAFT_ERROR, @@ -851,7 +851,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi std::this_thread::sleep_for(sleep_time * (i + 1)); }; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) { for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { @@ -863,7 +863,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (remove->id == state_manager->server_id()) { @@ -884,7 +884,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return; @@ -928,7 +928,7 @@ bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAc backoff(i); } } - else if (std::get_if(&action) != nullptr) + else if (std::holds_alternative(action)) return true; return false; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 61e29b67bbd..50d229c9e63 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -17,8 +17,6 @@ namespace DB using RaftAppendResult = nuraft::ptr>>; -class KeeperDispatcher; - class KeeperServer { private: diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3e9850caa40..6ec03235a2d 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,15 +16,13 @@ #include #include #include -#include "Coordination/KeeperStorage.h" -#include "Coordination/KeeperReconfiguration.h" - #include namespace ProfileEvents { extern const Event KeeperCommits; + extern const Event KeeperReconfigRequest; extern const Event KeeperCommitsFailed; extern const Event KeeperSnapshotCreations; extern const Event KeeperSnapshotCreationsFailed; @@ -298,6 +298,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { + ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); + const auto& request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; @@ -312,7 +314,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; - KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); if (!dispatcher.clusterUpdateQueueEmpty()) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 3b239adae45..4ff46394fcc 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,8 +12,6 @@ namespace DB { -class KeeperDispatcher; - using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 2b245a455b7..7fe85857ccb 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1088,7 +1088,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce if (request.path == Coordination::keeper_config_path) { response.data = serializeClusterConfig( - storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); response.error = Coordination::Error::ZOK; return response_ptr; } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 42923dd0b29..45b6d5d1dad 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -31,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) if (!id_str.starts_with("server.")) return std::nullopt; - int id; + Int32 id; if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) return std::nullopt; if (id <= 0) From 297d566600c3b36a552b456f8371440c5939b1d7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 14:13:26 +0000 Subject: [PATCH 220/871] acl check --- src/Coordination/KeeperStateMachine.cpp | 3 +++ src/Coordination/KeeperStateMachine.h | 5 +++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6ec03235a2d..e053e481b6b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -314,6 +314,9 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; + if (!storage->checkACL(keeper_config_path, Coordination::ACL::Write, session_id, true)) + return bad_request(ZNOAUTH); + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 4ff46394fcc..997a03a04d5 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -183,7 +183,8 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; - KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); + KeeperStorage::ResponseForSession processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock); }; - } From 1bef6fc76cf70b6faeb82b25e53e708bbf309bc6 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 15:04:31 +0000 Subject: [PATCH 221/871] process reconfig in keeper dispatcher --- src/Coordination/KeeperDispatcher.cpp | 6 ++++++ src/Coordination/KeeperStateMachine.cpp | 21 +++++++++++++-------- src/Coordination/KeeperStateMachine.h | 2 ++ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 26be2881780..9039b3a6d11 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -90,6 +90,12 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + { + server->getKeeperStateMachine()->reconfigure(request); + continue; + } + KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e053e481b6b..c837b93ffdd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -295,6 +295,19 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) +{ + std::lock_guard _(storage_and_responses_lock); + KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } +} + KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { @@ -400,14 +413,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response->session_id = session_id; try_push(response_for_session); } - // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies - // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. - // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details - else if (op_num == Coordination::OpNum::Reconfig) - { - std::lock_guard lock(storage_and_responses_lock); - try_push(processReconfiguration(*request_for_session)); - } else { if (op_num == Coordination::OpNum::Close) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 997a03a04d5..5762476886c 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -126,6 +126,8 @@ public: void recalculateStorageStats(); + void reconfigure(const KeeperStorage::RequestForSession& request_for_session); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored From 4550b15876c7e57533e7aa700b1376682c95de69 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 16:38:26 +0000 Subject: [PATCH 222/871] try updating tests to wait for cluster configs to come in sync --- tests/integration/helpers/keeper_utils.py | 14 +++++++++++--- tests/integration/test_keeper_reconfig_add/test.py | 4 ++-- .../test_keeper_reconfig_remove/test.py | 6 +++--- .../test_keeper_reconfig_remove_many/test.py | 10 +++++----- .../test_keeper_reconfig_replace_leader/test.py | 10 +++++----- .../test.py | 10 +++++----- 6 files changed, 31 insertions(+), 23 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3da1d5bd7b0..3970aa325ad 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -77,8 +77,16 @@ def get_config_str(zk: KazooClient) -> str: return zk.get("/keeper/config")[0].decode("utf-8") -def configs_equal(left: str, right: str) -> bool: +def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = 30.0): """ - Check whether /keeper/config nodes are equal + Check whether get /keeper/config result in left_config is equal + to get /keeper/config on right_zk ZK connection. """ - return sorted(left.split("\n")) == sorted(right.split("\n")) + elapsed: float = 0. + while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + time.sleep(1) + elapsed += 1 + if elapsed >= timeout: + raise Exception( + f"timeout while checking nodes configs to get equal. " + f"Left: {left_config}, right: {get_config_str(right_zk)}") diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py index c80279a0727..2c2da7403a1 100644 --- a/tests/integration/test_keeper_reconfig_add/test.py +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -91,7 +91,7 @@ def test_reconfig_add(started_cluster): assert "node3" not in config zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"/test_three_{i}") is not None @@ -132,7 +132,7 @@ def test_reconfig_add(started_cluster): assert "node3" in config zk3 = get_fake_zk(node3) - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk3.exists(f"/test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py index 7f0b1ee92c6..fb0a9472df3 100644 --- a/tests/integration/test_keeper_reconfig_remove/test.py +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -70,11 +70,11 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None @@ -92,7 +92,7 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2.stop() zk2.close() zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py index 6bf477ff9c9..ec0d8b95eff 100644 --- a/tests/integration/test_keeper_reconfig_remove_many/test.py +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -54,11 +54,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) zk5 = get_fake_zk(node5) zk5.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk5)) + ku.wait_configs_equal(config, zk5) for i in range(100): assert zk4.exists(f"test_two_{i}") is not None @@ -83,7 +83,7 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk1 = get_fake_zk(node1) zk1.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk1)) + ku.wait_configs_equal(config, zk1) for i in range(200): assert zk1.exists(f"test_two_{i}") is not None @@ -128,11 +128,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_leader_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index 1b23aa056c6..ca1ec3a0c92 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -104,7 +104,7 @@ def test_reconfig_replace_leader(started_cluster): assert "node4" in config zk4 = get_fake_zk(node4) - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -114,13 +114,13 @@ def test_reconfig_replace_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index c7aed945097..76aed1c7f3a 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -91,7 +91,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -107,13 +107,13 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From 2f0cd054970015799b394588b7ecf79ca34a6e9a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:35:25 +0000 Subject: [PATCH 223/871] handle leader removal corner cases --- src/Coordination/KeeperServer.cpp | 7 +++++-- .../test.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f6715b0da3f..fd82f220f9b 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -798,9 +798,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) || raft_instance->add_srv(static_cast(*add))->get_accepted(); else if (const auto * remove = std::get_if(&action)) { - if (isLeader() && remove->id == state_manager->server_id()) + if (remove->id == raft_instance->get_leader()) { - raft_instance->yield_leadership(); + if (isLeader()) + raft_instance->yield_leadership(); + else + raft_instance->request_leadership(); return false; } diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 76aed1c7f3a..e23d0674c12 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -113,7 +113,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - ku.configs_equal(config, zk3) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From c2a0607cf890f95e94db2751a3e68b7acc59a5bf Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:52:57 +0000 Subject: [PATCH 224/871] fix --- tests/integration/helpers/keeper_utils.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3970aa325ad..93ea3fa74b7 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -82,11 +82,14 @@ def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = Check whether get /keeper/config result in left_config is equal to get /keeper/config on right_zk ZK connection. """ - elapsed: float = 0. - while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + elapsed: float = 0.0 + while sorted(left_config.split("\n")) != sorted( + get_config_str(right_zk).split("\n") + ): time.sleep(1) elapsed += 1 if elapsed >= timeout: raise Exception( f"timeout while checking nodes configs to get equal. " - f"Left: {left_config}, right: {get_config_str(right_zk)}") + f"Left: {left_config}, right: {get_config_str(right_zk)}" + ) From c46b125d0a8501241a4a726a32141e1215a2cbf5 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 5 Jul 2023 18:23:34 +0000 Subject: [PATCH 225/871] review fixes --- src/Coordination/KeeperDispatcher.cpp | 2 ++ src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 5 ++++- src/Coordination/tests/gtest_coordination.cpp | 10 +++++----- utils/keeper-data-dumper/main.cpp | 2 +- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9039b3a6d11..90996dfaff7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -121,6 +121,8 @@ void KeeperDispatcher::requestThread() std::lock_guard lock(read_request_queue_mutex); read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } + else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + server->getKeeperStateMachine()->reconfigure(request); else { current_batch_bytes_size += request.request->bytesSize(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c837b93ffdd..45c776e105b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -313,7 +313,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); - const auto& request = static_cast(*request_for_session.request); + const auto & request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 5762476886c..116fa9257a0 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -88,7 +88,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + // This should be used only for tests or keeper-data-dumper because it violates + // TSA -- we can't acquire the lock outside of this class or return a storage under lock + // in a reasonable way. + KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS { return *storage; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index b302f9b13ca..03ce23e9233 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1616,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorageForUnitTests(); - auto & restored_storage = restore_machine->getStorageForUnitTests(); + auto & source_storage = state_machine->getStorageUnsafe(); + auto & restored_storage = restore_machine->getStorageUnsafe(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1719,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorageForUnitTests(); + const auto & storage = state_machine->getStorageUnsafe(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1768,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1831,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 8d685d65d1d..22e5f47687a 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorageForUnitTests(); + auto & storage = machine->getStorageUnsafe(); std::queue keys; keys.push("/"); From bafcc3afdc79463915b53b5e441758cbb958b958 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 15:18:49 +0000 Subject: [PATCH 226/871] remove reconfig in process flag as it's useless --- src/Common/ZooKeeper/IKeeper.h | 1 - src/Coordination/KeeperDispatcher.cpp | 5 ----- src/Coordination/KeeperDispatcher.h | 1 - src/Coordination/KeeperStateMachine.cpp | 4 +--- src/Coordination/RaftServerConfig.cpp | 9 ++++----- 5 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 8567a53699e..5240acc2616 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,7 +82,6 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state - ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 90996dfaff7..b956bba4031 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -780,11 +780,6 @@ void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) } } -bool KeeperDispatcher::clusterUpdateQueueEmpty() const -{ - return cluster_update_queue.empty(); -} - bool KeeperDispatcher::reconfigEnabled() const { return server->reconfigEnabled(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index a9b3d33eb51..40f1dac1570 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -136,7 +136,6 @@ public: void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void pushClusterUpdates(ClusterUpdateActions&& actions); - bool clusterUpdateQueueEmpty() const; bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 45c776e105b..b821050cccf 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -309,7 +309,7 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req } KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) + const KeeperStorage::RequestForSession & request_for_session) { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); @@ -333,8 +333,6 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); - if (!dispatcher.clusterUpdateQueueEmpty()) - return bad_request(ZRECONFIGINPROGRESS); if (request.version != -1) return bad_request(ZBADVERSION); diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 45b6d5d1dad..9090ed68fb6 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -1,7 +1,6 @@ #include "RaftServerConfig.h" -#include -#include #include +#include #include namespace DB @@ -32,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return std::nullopt; Int32 id; - if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + if (!tryParse(id, std::next(id_str.begin(), 7))) return std::nullopt; if (id <= 0) return std::nullopt; @@ -44,7 +43,7 @@ std::optional RaftServerConfig::parse(std::string_view server) const std::string_view port = endpoint.substr(port_delimiter + 1); uint16_t port_tmp; - if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + if (!tryParse(port_tmp, port)) return std::nullopt; RaftServerConfig out{id, endpoint}; @@ -59,7 +58,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return out; const std::string_view priority = parts[3]; - if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + if (!tryParse(out.priority, priority)) return std::nullopt; if (out.priority < 0) return std::nullopt; From 5302b478a4b512d080068563d4b5b983e4b13d77 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 17:12:24 +0000 Subject: [PATCH 227/871] proper reconfig batch handling --- src/Common/ZooKeeper/IKeeper.cpp | 1 - src/Coordination/KeeperDispatcher.cpp | 32 +++++++++++++++++---------- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 50160279506..f0a07241735 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,7 +110,6 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; - case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b956bba4031..daa65de0d89 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -82,6 +82,7 @@ void KeeperDispatcher::requestThread() /// requests into a batch we must check that the new request is not read request. Otherwise we have to /// process all already accumulated write requests, wait them synchronously and only after that process /// read request. So reads are some kind of "separator" for writes. + /// Also there is a special reconfig request also being a separator. try { if (requests_queue->tryPop(request, max_wait)) @@ -90,20 +91,17 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - { - server->getKeeperStateMachine()->reconfigure(request); - continue; - } - KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; bool has_read_request = false; + bool has_reconfig_request = false; - /// If new request is not read request or we must to process it through quorum. + /// If new request is not read request or reconfig request we must process it through quorum. /// Otherwise we will process it locally. - if (coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + has_reconfig_request = true; + else if (coordination_settings->quorum_reads || !request.request->isReadRequest()) { current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); @@ -122,7 +120,10 @@ void KeeperDispatcher::requestThread() read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - server->getKeeperStateMachine()->reconfigure(request); + { + has_reconfig_request = true; + return false; + } else { current_batch_bytes_size += request.request->bytesSize(); @@ -138,6 +139,7 @@ void KeeperDispatcher::requestThread() /// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; while (!shutdown_called && !has_read_request && + !has_reconfig_request && current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size && try_get_request()) ; @@ -150,8 +152,10 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && !prev_result_done() && - current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size) + while (!shutdown_called && !has_read_request && + !has_reconfig_request && !prev_result_done() && + current_batch.size() <= max_batch_size + && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); } @@ -175,7 +179,8 @@ void KeeperDispatcher::requestThread() if (result) { - if (has_read_request) /// If we will execute read request next, than we have to process result now + /// If we will execute read or reconfig next, we have to process result now + if (has_read_request || has_reconfig_request) forceWaitAndProcessResult(result, current_batch); } else @@ -189,6 +194,9 @@ void KeeperDispatcher::requestThread() prev_result = result; } + if (has_reconfig_request) + server->getKeeperStateMachine()->reconfigure(request); + /// Read request always goes after write batch (last request) if (has_read_request) { From ef3551fea00b6eeaa76884880a977e9a0768bb82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Jul 2023 03:54:10 +0200 Subject: [PATCH 228/871] Maybe better tests --- tests/queries/0_stateless/00995_exception_while_insert.sh | 3 +-- tests/queries/0_stateless/01030_limit_by_with_ties_error.sh | 3 +-- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 +- tests/queries/0_stateless/01442_merge_detach_attach_long.sh | 3 +-- tests/queries/0_stateless/01515_logtrace_function.sh | 3 +-- .../01583_parallel_parsing_exception_with_offset.sh | 3 +-- tests/queries/0_stateless/02359_send_logs_source_regexp.sh | 2 +- .../0_stateless/02360_rename_table_along_with_log_name.sh | 2 +- 8 files changed, 8 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00995_exception_while_insert.sh b/tests/queries/0_stateless/00995_exception_while_insert.sh index e0cd264a2b7..927ac6a54e5 100755 --- a/tests/queries/0_stateless/00995_exception_while_insert.sh +++ b/tests/queries/0_stateless/00995_exception_while_insert.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh index 711a015f044..c3414838789 100755 --- a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh +++ b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query=""" SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10) ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index dacb609d790..fccac57aea8 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -4,13 +4,13 @@ unset CLICKHOUSE_LOG_COMMENT CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index c080dded1c8..acb2550d48c 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -4,11 +4,10 @@ set -e CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" diff --git a/tests/queries/0_stateless/01515_logtrace_function.sh b/tests/queries/0_stateless/01515_logtrace_function.sh index 131ec0edb9e..4ebecd0cc18 100755 --- a/tests/queries/0_stateless/01515_logtrace_function.sh +++ b/tests/queries/0_stateless/01515_logtrace_function.sh @@ -2,9 +2,8 @@ # Tags: race CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=debug # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') - ${CLICKHOUSE_CLIENT} --query="SELECT logTrace('logTrace Function Test');" 2>&1 | grep -q "logTrace Function Test" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh index aa3a25096c0..00d22cb8e83 100755 --- a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh +++ b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh index d3b60bc59f4..f287e323ca7 100755 --- a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh +++ b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh [ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="executeQuery|InterpreterSelectQuery" $CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2) diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh index e8c7f844b5c..c07dcdd549b 100755 --- a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -11,7 +12,6 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" $CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" $CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" -CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name $CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" From cd3080428ea3da6a71169c929e959a0c3f9c5d5b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:58:01 +0000 Subject: [PATCH 229/871] Fix async connect to hosts with multiple ips --- src/Client/Connection.cpp | 2 + src/Client/Connection.h | 4 ++ src/Client/ConnectionEstablisher.cpp | 7 +- src/Client/ConnectionEstablisher.h | 2 + .../configs/enable_hedged.xml | 8 +++ .../configs/listen_host.xml | 4 ++ .../test.py | 65 +++++++++++++++++++ 7 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index cd102f46ffe..cac5600fbcb 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -105,6 +105,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) for (auto it = addresses.begin(); it != addresses.end();) { + have_more_addresses_to_connect = it != std::prev(addresses.end()); + if (connected) disconnect(); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index cb3f2507cb9..f4daf8e3aeb 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -159,6 +159,8 @@ public: out->setAsyncCallback(async_callback); } + bool haveMoreAddressesToConnect() const { return have_more_addresses_to_connect; } + private: String host; UInt16 port; @@ -227,6 +229,8 @@ private: std::shared_ptr maybe_compressed_out; std::unique_ptr block_out; + bool have_more_addresses_to_connect = false; + /// Logger is created lazily, for avoid to run DNS request in constructor. class LoggerWrapper { diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 897fb5fde73..439025447ca 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -179,7 +179,7 @@ bool ConnectionEstablisherAsync::checkTimeout() is_timeout_alarmed = true; } - if (is_timeout_alarmed && !is_socket_ready) + if (is_timeout_alarmed && !is_socket_ready && !haveMoreAddressesToConnect()) { /// In not async case timeout exception would be thrown and caught in ConnectionEstablisher::run, /// but in async case we process timeout outside and cannot throw exception. So, we just save fail message. @@ -225,6 +225,11 @@ void ConnectionEstablisherAsync::resetResult() } } +bool ConnectionEstablisherAsync::haveMoreAddressesToConnect() +{ + return !result.entry.isNull() && result.entry->haveMoreAddressesToConnect(); +} + #endif } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 5b58563dc01..a8126900d3b 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -104,6 +104,8 @@ private: void resetResult(); + bool haveMoreAddressesToConnect(); + ConnectionEstablisher connection_establisher; TryResult result; std::string fail_message; diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml new file mode 100644 index 00000000000..238370176af --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -0,0 +1,8 @@ + + + + 1 + 0 + + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml new file mode 100644 index 00000000000..df0247fd651 --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml @@ -0,0 +1,4 @@ + + :: + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py new file mode 100644 index 00000000000..0c18a316d4b --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -0,0 +1,65 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def cluster_without_dns_cache_update(): + try: + cluster.start() + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + pass + + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.11", +) + +node2 = cluster.add_instance( + "node2", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.12", +) + +# node1 - source with table, have invalid ipv6 +# node2 - destination, doing remote query +def test(cluster_without_dns_cache_update): + node1.query( + "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" + ) + node1.query( + "INSERT INTO test SELECT toDate('2022-12-28'), 1;" + ) + assert node1.query( + 'SELECT count(*) FROM test' + ) == '1\n' + + wrong_ip = '2001:3984:3989::1:1118' + + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) + ) + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ) + + assert node1.query("SELECT count(*) from test") == "1\n" + node2.query("SYSTEM DROP DNS CACHE") + node1.query("SYSTEM DROP DNS CACHE") + assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" + From fc94cc8b87fceb8b6631b72e34a6c10fdc197f83 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:59:27 +0000 Subject: [PATCH 230/871] Update config for test --- .../test_async_connect_to_multiple_ips/configs/enable_hedged.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml index 238370176af..399d886ee6a 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -2,7 +2,6 @@ 1 - 0 From e618dd05cc73b7ad38296e7c28f66b6f077343f8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 7 Jul 2023 13:03:44 +0200 Subject: [PATCH 231/871] 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 58cf3e5d210..4b47da9affb 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 0f5a52b275c..504cf0326f0 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 232/871] 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 ca8a412bf2e..bcd3244b5a9 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 48adf36e678..d3d8c0f2bc8 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 39d0b309bd730748b52acfb32de729e8f8496f83 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 7 Jul 2023 13:15:26 +0000 Subject: [PATCH 233/871] Make own function with slices --- src/Functions/HasSubsequenceImpl.h | 187 ++++++++---------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 2 +- src/Functions/hasSubsequenceUTF8.cpp | 2 +- 5 files changed, 84 insertions(+), 111 deletions(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index bcb8e8e99e6..1396e64ade5 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,124 +1,109 @@ #pragma once + +#include +#include +#include +#include +#include namespace DB { namespace { -template -struct HasSubsequenceImpl -{ - using ResultType = UInt8; - static constexpr bool use_default_implementation_for_constants = false; - static constexpr bool supports_start_pos = false; +using namespace GatherUtils; + +template +class FunctionsHasSubsequenceImpl : public IFunction +{ +public: static constexpr auto name = Name::name; - static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + static FunctionPtr create(ContextPtr) { return std::make_shared(); } - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const String & needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * /*res_null*/) + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {};} + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (needle.empty()) - { - for (auto & r : res) - r = 1; - return; - } + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); - ColumnString::Offset prev_haystack_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) - { - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); - prev_haystack_offset = haystack_offsets[i]; - } + if (!isString(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); + + return std::make_shared>(); } - static void vectorVector( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - ColumnString::Offset prev_haystack_offset = 0; - ColumnString::Offset prev_needle_offset = 0; + const ColumnPtr & column_haystack = arguments[0].column; + const ColumnPtr & column_needle = arguments[1].column; - size_t size = haystack_offsets.size(); + const ColumnConst * haystack_const_string = checkAndGetColumnConst(column_haystack.get()); + const ColumnConst * needle_const_string = checkAndGetColumnConst(column_needle.get()); + const ColumnString * haystack_string = checkAndGetColumn(&*column_haystack); + const ColumnString * needle_string = checkAndGetColumn(&*column_needle); - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(input_rows_count); - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); - } + if (haystack_string && needle_string) + execute(StringSource{*haystack_string}, StringSource{*needle_string}, vec_res); + else if (haystack_string && needle_const_string) + execute(StringSource{*haystack_string}, ConstSource{*needle_const_string}, vec_res); + else if (haystack_const_string && needle_string) + execute(ConstSource{*haystack_const_string}, StringSource{*needle_string}, vec_res); + else if (haystack_const_string && needle_const_string) + execute(ConstSource{*haystack_const_string}, ConstSource{*needle_const_string}, vec_res); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}, first argument of function {} must be a string", + arguments[0].column->getName(), + getName()); - prev_haystack_offset = haystack_offsets[i]; - prev_needle_offset = needle_offsets[i]; - } + return col_res; } - static void constantVector( - const String & haystack, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) +private: + + template + void execute( + SourceHaystack && haystacks, + SourceNeedle && needles, + PaddedPODArray & res_data) const { - ColumnString::Offset prev_needle_offset = 0; + size_t row_num = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + while (!haystacks.isEnd()) { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + [[maybe_unused]] auto haystack_slice = haystacks.getWhole(); + [[maybe_unused]] auto needle_slice = needles.getWhole(); - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); - } - prev_needle_offset = needle_offsets[i]; - } - } + auto haystack = std::string(reinterpret_cast(haystack_slice.data), haystack_slice.size); + auto needle = std::string(reinterpret_cast(needle_slice.data), needle_slice.size); - static void constantConstant( - String haystack, - String needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) - { - size_t size = res.size(); - Impl::toLowerIfNeed(haystack); - Impl::toLowerIfNeed(needle); + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); - UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); - - for (size_t i = 0; i < size; ++i) - { - res[i] = result; + res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + haystacks.next(); + needles.next(); + ++row_num; } } @@ -130,18 +115,6 @@ struct HasSubsequenceImpl ++j; return j == needle_size; } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } - - template - static void vectorFixedVector(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } }; } diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index bb1f295cee4..900e80f5524 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequence static constexpr auto name = "hasSubsequence"; }; -using FunctionHasSubsequence = FunctionsStringSearch>; +using FunctionHasSubsequence = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequence) diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index fe50ada9be9..dbac62d7f09 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitive static constexpr auto name = "hasSubsequenceCaseInsensitive"; }; -using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitive = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index 2908c284a25..c104ff52857 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitiveUTF8 static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; }; -using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index c0811de6575..c67ce7d9c74 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequenceUTF8 static constexpr auto name = "hasSubsequenceUTF8"; }; -using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceUTF8) From ed37b01b515ea204223dd03cee5482ee6faad351 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 14:39:11 +0000 Subject: [PATCH 234/871] Fix style --- .../__init__.py | 0 .../test.py | 29 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/__init__.py diff --git a/tests/integration/test_async_connect_to_multiple_ips/__init__.py b/tests/integration/test_async_connect_to_multiple_ips/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py index 0c18a316d4b..acc4d24d0fa 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/test.py +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -36,30 +36,37 @@ node2 = cluster.add_instance( ipv4_address="10.5.95.12", ) + # node1 - source with table, have invalid ipv6 # node2 - destination, doing remote query def test(cluster_without_dns_cache_update): node1.query( "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" ) - node1.query( - "INSERT INTO test SELECT toDate('2022-12-28'), 1;" - ) - assert node1.query( - 'SELECT count(*) FROM test' - ) == '1\n' - - wrong_ip = '2001:3984:3989::1:1118' + node1.query("INSERT INTO test SELECT toDate('2022-12-28'), 1;") + assert node1.query("SELECT count(*) FROM test") == "1\n" + + wrong_ip = "2001:3984:3989::1:1118" node2.exec_in_container( (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) ) node2.exec_in_container( - (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ( + [ + "bash", + "-c", + "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name), + ] + ) ) assert node1.query("SELECT count(*) from test") == "1\n" node2.query("SYSTEM DROP DNS CACHE") node1.query("SYSTEM DROP DNS CACHE") - assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" - + assert ( + node2.query( + f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;" + ) + == "1\n" + ) From 39a440fa0e99849d710e09bd031de5a52708fd6f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jul 2023 22:08:31 +0000 Subject: [PATCH 235/871] 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 32bfa6935df..3d1e2650188 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 8e4c8f118cf64fcd77524439508b838c05a58fcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Jul 2023 09:07:05 +0200 Subject: [PATCH 236/871] Fix disaster in integration tests, part 2 --- tests/integration/ci-runner.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index d6d17abe725..43184574e6e 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -406,9 +406,9 @@ class ClickhouseIntegrationTestsRunner: out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( "cd {repo_path}/tests/integration && " - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " - "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " - "| grep -v 'SKIPPED' | sort -u > {out_file}".format( + "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "| tee '{out_file_full}' | grep -F '::' | sed -r 's/ \(fixtures used:.*//g; s/^ *//g; s/ *$//g' " + "| grep -v -F 'SKIPPED' | sort --unique > {out_file}".format( repo_path=repo_path, runner_opts=self._get_runner_opts(), image_cmd=image_cmd, @@ -626,7 +626,7 @@ class ClickhouseIntegrationTestsRunner: info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = " ".join([test for test in sorted(test_names)]) + test_cmd = " ".join([f"'{test}'" for test in sorted(test_names)]) parallel_cmd = ( " --parallel {} ".format(num_workers) if num_workers > 0 else "" ) From 62bfa4ed93fb3796eccb0df041a9dfa057583c9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 02:21:48 +0200 Subject: [PATCH 237/871] Fix performance test for regexp cache --- src/Functions/Regexps.h | 4 +++- tests/performance/re2_regex_caching.xml | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 4bfd10bdbf5..aa8ae5b4054 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -21,6 +22,7 @@ # include #endif + namespace ProfileEvents { extern const Event RegexpCreated; @@ -86,7 +88,7 @@ public: private: constexpr static size_t CACHE_SIZE = 100; /// collision probability - std::hash hasher; + DefaultHash hasher; struct Bucket { String pattern; /// key diff --git a/tests/performance/re2_regex_caching.xml b/tests/performance/re2_regex_caching.xml index 6edc83097ba..9778a8d4c0c 100644 --- a/tests/performance/re2_regex_caching.xml +++ b/tests/performance/re2_regex_caching.xml @@ -24,8 +24,8 @@ '.*' || toString(number) || '.' '.*' || toString(number % 10) || '.' - - '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number) + + '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10) 0 + 0 From 6fd27b6cd882b31f73ecd27ca7ae0bb2f0d25854 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 22:19:35 +0200 Subject: [PATCH 457/871] Fix build --- src/Storages/StorageMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 085d532b09c..32e100edc4d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -45,6 +45,7 @@ #include #include + namespace DB { @@ -940,7 +941,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; - auto is_background_memory_usage_ok = [](String * disable_reason) -> bool + auto is_background_memory_usage_ok = [](String & disable_reason) -> bool { if (canEnqueueBackgroundTask()) return true; From ff6e5ff1c547494ed7c6320c5d62bf789d433ae2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jul 2023 20:23:55 +0000 Subject: [PATCH 458/871] 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 f6ba3834c92..96264e53522 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -7,10 +7,16 @@ from helpers.test_tools import assert_eq_with_retry, TSV 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 71f8b9a759d..074667fc92f 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 @@ from helpers.cluster import ClickHouseCluster 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 3715c7f461dc9a0c48ea3cfac52ef52c47a53c64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:08:14 +0200 Subject: [PATCH 459/871] Fix error in a test --- tests/queries/0_stateless/02293_selected_rows_and_merges.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh index 76c562c9744..2f281d27814 100755 --- a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh +++ b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh @@ -24,4 +24,4 @@ ${CLICKHOUSE_CLIENT} -q "system flush logs" # Here for mutation all values are 0, cause mutation is executed async. # It's pretty hard to write a test with total counter. -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'] > 1000, ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" From c724816cb8403c07d2d4c4601e0c4c9dcfc16e5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:15:16 +0200 Subject: [PATCH 460/871] Fix test --- .../configs/config.d/merge_tree.xml | 5 +++++ .../configs/config.d/users.xml | 5 ----- .../configs/config.xml | 22 ------------------- .../test_merge_tree_s3_failover/test.py | 1 + 4 files changed, 6 insertions(+), 27 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml delete mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml delete mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.xml diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml new file mode 100644 index 00000000000..c58c957b596 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml @@ -0,0 +1,5 @@ + + + 1.0 + + diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml deleted file mode 100644 index 0011583a68c..00000000000 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - - - diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml deleted file mode 100644 index 743d75d9a21..00000000000 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ /dev/null @@ -1,22 +0,0 @@ - - 9000 - 127.0.0.1 - - - - true - none - - AcceptCertificateHandler - - - - - 500 - ./clickhouse/ - users.xml - - - 1.0 - - diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 90dda631924..57ca5ed5ffd 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -67,6 +67,7 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/instant_moves.xml", "configs/config.d/part_log.xml", + "configs/config.d/merge_tree.xml" ], with_minio=True, ) From 3c8141529f0f8d4d7c48c077e91af77ee9885ad8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jul 2023 23:25:21 +0000 Subject: [PATCH 461/871] Automatic style fix --- tests/integration/test_merge_tree_s3_failover/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 57ca5ed5ffd..b47d741e78e 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -67,7 +67,7 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/instant_moves.xml", "configs/config.d/part_log.xml", - "configs/config.d/merge_tree.xml" + "configs/config.d/merge_tree.xml", ], with_minio=True, ) From a19a1001f063ce6d992ffc08d6d05d3ef7342b66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:28:22 +0200 Subject: [PATCH 462/871] Fix 01111_create_drop_replicated_db_stress --- .../01111_create_drop_replicated_db_stress.sh | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) 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 4d341e5b8a3..cc63af3676b 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 @@ -8,7 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function create_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do SHARD=$(($RANDOM % 2)) REPLICA=$(($RANDOM % 2)) SUFFIX=$(($RANDOM % 16)) @@ -24,7 +25,8 @@ function create_db() function drop_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [[ "$database" == "$CLICKHOUSE_DATABASE" ]]; then continue; fi if [ -z "$database" ]; then continue; fi @@ -36,7 +38,8 @@ function drop_db() function sync_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --receive_timeout=1 -q \ @@ -47,7 +50,8 @@ function sync_db() function create_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -59,7 +63,8 @@ function create_table() function alter_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -71,7 +76,8 @@ function alter_table() function insert() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT -q \ @@ -81,23 +87,16 @@ function insert() -export -f create_db -export -f drop_db -export -f sync_db -export -f create_table -export -f alter_table -export -f insert - TIMEOUT=30 -timeout $TIMEOUT bash -c create_db & -timeout $TIMEOUT bash -c sync_db & -timeout $TIMEOUT bash -c create_table & -timeout $TIMEOUT bash -c alter_table & -timeout $TIMEOUT bash -c insert & +create_db $TIMEOUT & +sync_db $TIMEOUT & +create_table $TIMEOUT & +alter_table $TIMEOUT & +insert $TIMEOUT & sleep 1 # give other queries a head start -timeout $TIMEOUT bash -c drop_db & +drop_db $TIMEOUT & wait From 6d915042a23ed0fd320b98118d9527e312d49ffe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:44:20 +0200 Subject: [PATCH 463/871] Fix ugly code --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 4beffbcf869..909133dfa4a 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field({}) in arrow schema:{}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); else continue; } @@ -168,4 +168,3 @@ private: }; } #endif - From 0789f388c3f6acbfdb42f44ee6463b3d646ddc27 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 02:45:56 +0300 Subject: [PATCH 464/871] Update ArrowFieldIndexUtil.h --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 909133dfa4a..b7adaa35335 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the following Arrow schema:\n{}\n", named_col.name, schema.ToString()); else continue; } From d666272b7666967cf1d1bed3804673e3beb1ca64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 05:29:12 +0200 Subject: [PATCH 465/871] Enable `allow_vertical_merges_from_compact_to_wide_parts` by default --- src/Storages/MergeTree/MergeTreeSettings.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dc24327712c..783fde088dc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -160,7 +160,7 @@ struct Settings; M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ - M(Bool, allow_vertical_merges_from_compact_to_wide_parts, false, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ + M(Bool, allow_vertical_merges_from_compact_to_wide_parts, true, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ M(UInt64, zero_copy_merge_mutation_min_parts_size_sleep_before_lock, 1ULL * 1024 * 1024 * 1024, "If zero copy replication is enabled sleep random amount of time before trying to lock depending on parts size for merge or mutation", 0) \ \ @@ -169,8 +169,9 @@ struct Settings; M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ M(Bool, use_metadata_cache, false, "Experimental feature to speed up parts loading process by using MergeTree metadata cache", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ - M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ + M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ M(Bool, compress_primary_key, true, "Primary key support compression, reduce primary key file size and speed up network transmission.", 0) \ From c3b8978023fae8adaa98a111f6253be50ee72a35 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 11:53:03 +0800 Subject: [PATCH 466/871] Don't use minmax_count projections when counting nullable columns --- .../optimizeUseAggregateProjection.cpp | 32 ++++--------------- ..._count_projection_count_nullable.reference | 1 + ...minmax_count_projection_count_nullable.sql | 9 ++++++ 3 files changed, 17 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference create mode 100644 tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..4f25118958f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -92,18 +92,6 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( return info; } -static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names) -{ - for (const auto & query_name : names) - { - auto jt = index.find(query_name); - if (jt == index.end() || jt->second->result_type->isNullable()) - return true; - } - - return false; -} - struct AggregateFunctionMatch { const AggregateDescription * description = nullptr; @@ -170,20 +158,14 @@ std::optional matchAggregateFunctions( } /// This is a special case for the function count(). - /// We can assume that 'count(expr) == count()' if expr is not nullable. - if (typeid_cast(candidate.function.get())) + /// We can assume that 'count(expr) == count()' if expr is not nullable, + /// which can be verified by simply casting to `AggregateFunctionCount *`. + if (typeid_cast(aggregate.function.get())) { - bool has_nullable_or_missing_arg = false; - has_nullable_or_missing_arg |= hasNullableOrMissingColumn(query_index, aggregate.argument_names); - has_nullable_or_missing_arg |= hasNullableOrMissingColumn(proj_index, candidate.argument_names); - - if (!has_nullable_or_missing_arg) - { - /// we can ignore arguments for count() - found_match = true; - res.push_back({&candidate, DataTypes()}); - break; - } + /// we can ignore arguments for count() + found_match = true; + res.push_back({&candidate, DataTypes()}); + break; } /// Now, function names and types matched. diff --git a/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql new file mode 100644 index 00000000000..048d725e0a0 --- /dev/null +++ b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (`val` LowCardinality(Nullable(String))) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; + +insert into test select number == 3 ? 'some value' : null from numbers(5); + +SELECT count(val) FROM test SETTINGS optimize_use_implicit_projections = 1; + +DROP TABLE test; From 65de310137a4e192499119128aa069375eb007c8 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 06:15:57 +0000 Subject: [PATCH 467/871] Return back SystemLogBase --- src/Common/SystemLogBase.cpp | 40 +++++++++++++++++++++++++++++++++- src/Common/SystemLogBase.h | 33 ++++++++++++++++++++++++++++ src/Interpreters/SystemLog.cpp | 29 ++---------------------- src/Interpreters/SystemLog.h | 16 +++----------- 4 files changed, 77 insertions(+), 41 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 1d0673e30dd..baee7021c35 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -202,7 +202,45 @@ void SystemLogQueue::shutdown() flush_event.notify_all(); } -#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; +template +SystemLogBase::SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_) + : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) +{ +} + +template +void SystemLogBase::startup() +{ + std::lock_guard lock(queue->mutex); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); +} + +template +void SystemLogBase::add(const LogElement & element) +{ + queue->push(element); +} + +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } + +#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue; +SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) + } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 9436137d4a8..5718182e115 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -121,4 +121,37 @@ private: const size_t flush_interval_milliseconds; }; + + +template +class SystemLogBase : public ISystemLog +{ +public: + using Self = SystemLogBase; + + SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_ = nullptr); + + void startup() override; + + /** Append a record into log. + * Writing to table will be done asynchronously and in case of failure, record could be lost. + */ + void add(const LogElement & element); + + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. + void flush(bool force) override; + + /// Non-blocking flush data in the buffer to disk. + void notifyFlush(bool force); + + String getName() const override { return LogElement::name(); } + + static const char * getDefaultOrderBy() { return "event_date, event_time"; } + +protected: + std::shared_ptr> queue; +}; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3193baa551f..674210cbaad 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -334,23 +334,16 @@ SystemLog::SystemLog( const String & storage_def_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : WithContext(context_) + : Base(database_name_ + "." + table_name_, flush_interval_milliseconds_, queue_) + , WithContext(context_) , log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")")) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) - , queue(queue_ ? queue_ : std::make_shared>(database_name_ + "." + table_name_, flush_interval_milliseconds_)) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); } -template -void SystemLog::startup() -{ - std::lock_guard lock(queue->mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); -} - template void SystemLog::shutdown() { @@ -618,24 +611,6 @@ ASTPtr SystemLog::getCreateTableQuery() return create; } -template -void SystemLog::add(const LogElement & element) -{ - queue->push(element); -} - -template -void SystemLog::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - -template -void SystemLog::notifyFlush(bool force) { queue->notifyFlush(force); } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6f61e075b49..91fb7f49221 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -89,10 +89,11 @@ struct SystemLogs template -class SystemLog : public ISystemLog, private boost::noncopyable, WithContext +class SystemLog : public SystemLogBase, private boost::noncopyable, WithContext { public: using Self = SystemLog; + using Base = SystemLogBase; /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. @@ -110,23 +111,12 @@ public: size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); - void startup() override; /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. */ - void add(const LogElement & element); void shutdown() override; - String getName() const override { return LogElement::name(); } - static const char * getDefaultOrderBy() { return "event_date, event_time"; } - - /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; - - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - void stopFlushThread() override; protected: @@ -134,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using Base::queue; private: @@ -144,7 +135,6 @@ private: String create_query; String old_create_query; bool is_prepared = false; - std::shared_ptr> queue; /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. From ee0453ed00ab5ecb232557e29d4e1f6365d83cd0 Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:18:50 +0800 Subject: [PATCH 468/871] fix issue#50582 tests Reproduced issue #50582, which occurs when sorting column contains constants and triggers the FinishSortingTransform. --- .../02815_fix_not_found_constants_col_in_block.reference | 2 ++ .../02815_fix_not_found_constants_col_in_block.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference new file mode 100644 index 00000000000..f2d4d23d9e3 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference @@ -0,0 +1,2 @@ +\N 1 19000 +\N 1 19000 diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql new file mode 100644 index 00000000000..c56d59c72d6 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingMergeTree; +insert into t0 values (19000, 1); +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; From d601d86fad94250ca3b749baa4478679cd6e1973 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 07:22:25 +0000 Subject: [PATCH 469/871] Remove empty line --- src/Common/SystemLogBase.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 5718182e115..fa9f9b6f72e 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -122,7 +122,6 @@ private: }; - template class SystemLogBase : public ISystemLog { From 629e0e0269dc96f88f781eb8a0a711667d50c92b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 16:06:21 +0800 Subject: [PATCH 470/871] Fix projection analysis with primary key analysis --- .../QueryPlan/Optimizations/optimizeTree.cpp | 8 +++++--- .../QueryPlan/Optimizations/projectionsCommon.cpp | 3 ++- .../QueryPlan/Optimizations/projectionsCommon.h | 1 - ..._projection_with_query_plan_optimization.reference | 1 + ...normal_projection_with_query_plan_optimization.sql | 11 +++++++++++ 5 files changed, 19 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 6cb76d540f7..01d192bb1f3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -146,8 +146,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + if (optimization_settings.optimize_projection) { + /// Normal projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; @@ -164,9 +169,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*stack.back().node, nodes); stack.pop_back(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index cb76ffa84ba..7ddda29cad4 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -131,7 +131,8 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (prewhere_info->prewhere_actions) { appendExpression(prewhere_info->prewhere_actions); - if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + if (const auto * filter_expression + = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) filter_nodes.push_back(filter_expression); else return false; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 1e9ab67c8fe..35daccad115 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -38,7 +38,6 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea /// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. -/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. struct QueryDAG { ActionsDAGPtr dag; diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql new file mode 100644 index 00000000000..30306ec5637 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql @@ -0,0 +1,11 @@ +drop table if exists t; + +CREATE TABLE t (id UInt64, id2 UInt64, id3 UInt64, PROJECTION t_reverse (SELECT id, id2, id3 ORDER BY id2, id, id3)) ENGINE = MergeTree ORDER BY (id) settings index_granularity = 4; + +insert into t SELECT number, -number, number FROM numbers(10000); + +set max_rows_to_read = 4; + +select count() from t where id = 3; + +drop table t; From 549026f0ae8041ba40f4557922c480f2f07715bf Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 16:11:14 +0800 Subject: [PATCH 471/871] fix style error fix Trailing whitespaces --- src/Processors/Transforms/FinishSortingTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index dd61472bc37..066928446f2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -50,7 +50,6 @@ FinishSortingTransform::FinishSortingTransform( description_sorted_without_constants.push_back(column_description); } } - /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. for (const auto & column_sort_desc : description_sorted_without_constants) From a86baab88b4444d5bf34e529bb737817daa20096 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:49:30 +0000 Subject: [PATCH 472/871] Fix test_replicated_database 'node doesn't exist' flakiness --- .../test_replicated_database/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index b3ba8d4737f..17dd2adcde4 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -132,14 +132,15 @@ def test_create_replicated_table(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_simple_alter_table(started_cluster, engine): + database = f"test_simple_alter_table_{engine}" main_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) # test_simple_alter_table - name = "test_simple_alter_table.alter_test_{}".format(engine) + name = f"{database}.alter_test" main_node.query( "CREATE TABLE {} " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " @@ -187,10 +188,9 @@ def test_simple_alter_table(started_cluster, engine): # test_create_replica_after_delay competing_node.query( - "CREATE DATABASE IF NOT EXISTS test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica3');" + f"CREATE DATABASE IF NOT EXISTS {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica3');" ) - name = "test_simple_alter_table.alter_test_{}".format(engine) main_node.query("ALTER TABLE {} ADD COLUMN Added3 UInt32;".format(name)) main_node.query("ALTER TABLE {} DROP COLUMN AddedNested1;".format(name)) main_node.query("ALTER TABLE {} RENAME COLUMN Added1 TO AddedNested1;".format(name)) @@ -210,21 +210,23 @@ def test_simple_alter_table(started_cluster, engine): ) assert_create_query([main_node, dummy_node, competing_node], name, expected) - main_node.query("DROP DATABASE test_simple_alter_table SYNC") - dummy_node.query("DROP DATABASE test_simple_alter_table SYNC") - competing_node.query("DROP DATABASE test_simple_alter_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") + competing_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_delete_from_table(started_cluster, engine): + database = f"delete_from_table_{engine}" + main_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - name = "delete_from_table.delete_test_{}".format(engine) + name = f"{database}.delete_test" main_node.query( "CREATE TABLE {} " "(id UInt64, value String) " @@ -241,7 +243,7 @@ def test_delete_from_table(started_cluster, engine): table_for_select = name if not "Replicated" in engine: - table_for_select = "cluster('delete_from_table', {})".format(name) + table_for_select = f"cluster('{database}', {name})" for node in [main_node, dummy_node]: assert_eq_with_retry( node, @@ -249,8 +251,8 @@ def test_delete_from_table(started_cluster, engine): expected, ) - main_node.query("DROP DATABASE delete_from_table SYNC") - dummy_node.query("DROP DATABASE delete_from_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") def get_table_uuid(database, name): @@ -278,18 +280,18 @@ def fixture_attachable_part(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_attach(started_cluster, attachable_part, engine): + database = f"alter_attach_{engine}" main_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - name = "alter_attach_test_{}".format(engine) main_node.query( - f"CREATE TABLE alter_attach.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_attach_test (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - table_uuid = get_table_uuid("alter_attach", name) + table_uuid = get_table_uuid(database, "alter_attach_test") # Provide and attach a part to the main node shutil.copytree( attachable_part, @@ -298,146 +300,146 @@ def test_alter_attach(started_cluster, attachable_part, engine): f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0", ), ) - main_node.query(f"ALTER TABLE alter_attach.{name} ATTACH PART 'all_1_1_0'") + main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" else: - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "" - main_node.query("DROP DATABASE alter_attach SYNC") - dummy_node.query("DROP DATABASE alter_attach SYNC") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_part(started_cluster, engine): + database = f"alter_drop_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_part (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_drop_part.{table} DROP PART '{part_name}'") - assert main_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + dummy_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_drop_part DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" else: assert ( - dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "456\n" ) - main_node.query("DROP DATABASE alter_drop_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_detach_part(started_cluster, engine): + database = f"alter_detach_part_{engine}" main_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_detach_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_detach_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_detach (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_detach_part.{table} DETACH PART '{part_name}'") - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_detach_part' AND table='{table}'" + dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" if engine == "ReplicatedMergeTree": # The detach operation is still replicated at the table engine level assert dummy_node.query(detached_parts_query) == f"{part_name}\n" else: assert dummy_node.query(detached_parts_query) == "" - main_node.query("DROP DATABASE alter_detach_part SYNC") - dummy_node.query("DROP DATABASE alter_detach_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_detached_part(started_cluster, engine): + database = f"alter_drop_detached_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_detached_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_detached_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_detached (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (123)") main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (456)") + dummy_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (456)") dummy_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DROP DETACHED PART '{part_name}'" ) - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_drop_detached_part' AND table='{table}'" + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_drop_detached'" assert main_node.query(detached_parts_query) == "" assert dummy_node.query(detached_parts_query) == f"{part_name}\n" - main_node.query("DROP DATABASE alter_drop_detached_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_detached_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_partition(started_cluster, engine): + database = f"alter_drop_partition_{engine}" main_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) snapshotting_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - table = f"alter_drop_partition.alter_drop_{engine}" main_node.query( - f"CREATE TABLE {table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO {table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO {table} VALUES (456)") - snapshotting_node.query(f"INSERT INTO {table} VALUES (789)") + dummy_node.query(f"INSERT INTO {database}.alter_drop VALUES (456)") + snapshotting_node.query(f"INSERT INTO {database}.alter_drop VALUES (789)") main_node.query( - f"ALTER TABLE {table} ON CLUSTER alter_drop_partition DROP PARTITION ID 'all'", + f"ALTER TABLE {database}.alter_drop ON CLUSTER {database} DROP PARTITION ID 'all'", settings={"replication_alter_partitions_sync": 2}, ) assert ( main_node.query( - f"SELECT CounterID FROM clusterAllReplicas('alter_drop_partition', {table})" + f"SELECT CounterID FROM clusterAllReplicas('{database}', {database}.alter_drop)" ) == "" ) - assert dummy_node.query(f"SELECT CounterID FROM {table}") == "" - main_node.query("DROP DATABASE alter_drop_partition") - dummy_node.query("DROP DATABASE alter_drop_partition") - snapshotting_node.query("DROP DATABASE alter_drop_partition") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop") == "" + main_node.query(f"DROP DATABASE {database}") + dummy_node.query(f"DROP DATABASE {database}") + snapshotting_node.query(f"DROP DATABASE {database}") def test_alter_fetch(started_cluster): From 8b0fc8283460f5678e733cef9803937ff9913177 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 29 Apr 2023 21:23:55 +0000 Subject: [PATCH 473/871] test_for_basic_auth_registry - mock up --- .../runner/compose/docker_compose_kafka.yml | 10 +++++++++ tests/integration/helpers/cluster.py | 22 ++++++++++++------- .../secrets/password | 1 + .../secrets/schema_registry_jaas.conf | 5 +++++ .../test_format_avro_confluent/test.py | 18 ++++++++++----- 5 files changed, 43 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_format_avro_confluent/secrets/password create mode 100644 tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 7e34f4c114d..39247f1dd37 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -39,7 +39,17 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC + # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO + # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar + SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + volumes: + - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - kafka1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0614cbf0e0d..9b5b33b1968 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1159,10 +1159,13 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + def setup_kafka_cmd( + self, instance, env_variables, docker_compose_yml_dir + ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" self.base_cmd.extend( @@ -1498,6 +1501,7 @@ class ClickHouseCluster: with_kafka=False, with_kerberized_kafka=False, with_kerberos_kdc=False, + with_secrets=False, with_rabbitmq=False, with_nats=False, clickhouse_path_dir=None, @@ -1604,6 +1608,7 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, + with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -3135,6 +3140,7 @@ class ClickHouseInstance: with_nats, with_nginx, with_kerberized_hdfs, + with_secrets, with_mongo, with_meili, with_redis, @@ -3197,7 +3203,7 @@ class ClickHouseInstance: if clickhouse_path_dir else None ) - self.kerberos_secrets_dir = p.abspath(p.join(base_path, "secrets")) + self.secrets_dir = p.abspath(p.join(base_path, "secrets")) self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path @@ -3220,6 +3226,7 @@ class ClickHouseInstance: self.with_nats = with_nats self.with_nginx = with_nginx self.with_kerberized_hdfs = with_kerberized_hdfs + self.with_secrets = with_secrets self.with_mongo = with_mongo self.with_meili = with_meili self.with_redis = with_redis @@ -4217,17 +4224,16 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - if ( - self.with_kerberized_kafka - or self.with_kerberized_hdfs - or self.with_kerberos_kdc - ): + if self.with_secrets: if self.with_kerberos_kdc: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path + from_dir=self.secrets_dir + to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( - self.kerberos_secrets_dir, + self.secrets_dir, p.abspath(p.join(base_secrets_dir, "secrets")), dirs_exist_ok=True, ) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password new file mode 100644 index 00000000000..8903cf6edd6 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -0,0 +1 @@ +schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf new file mode 100644 index 00000000000..7d0e6e2bf35 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf @@ -0,0 +1,5 @@ +RealmFooBar { + org.eclipse.jetty.jaas.spi.PropertyFileLoginModule required + file="/etc/schema-registry/secrets/password" + debug="true"; +}; \ No newline at end of file diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 42b7ddce193..921fbdf5ef4 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -14,7 +14,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("dummy", with_kafka=True) + cluster.add_instance("dummy", with_kafka=True, with_secrets=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -39,10 +39,13 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None + input("Cluster created, press any key to destroy...") - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -62,7 +65,12 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - schema_registry_url = "http://{}:{}".format( + # schema_registry_url = "http://{}:{}@{}:{}".format( + # 'schemauser', 'letmein', + # started_cluster.schema_registry_host, 8081 + # ) + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', started_cluster.schema_registry_host, 8081 ) From 1564eace38072417bf2c188d7c0a0c0e55321626 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Thu, 7 Jul 2022 13:48:25 +0530 Subject: [PATCH 474/871] enable url-encoded basic auth to fetch avro schema in kafka --- .../Formats/Impl/AvroRowInputFormat.cpp | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 4cd73cb23b5..fe795608970 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -52,6 +52,8 @@ #include #include #include +#include +#include #include #include #include @@ -934,6 +936,29 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); + if (!url.getUserInfo().empty()) { + Poco::Net::HTTPCredentials http_credentials; + Poco::Net::HTTPBasicCredentials http_basic_credentials; + std::string decoded_username; + std::string decoded_password; + + http_credentials.fromUserInfo(url.getUserInfo()); + + if (!http_credentials.getPassword().empty()) { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + Poco::URI::decode(http_credentials.getPassword(), decoded_password); + + http_basic_credentials.setUsername(decoded_username); + http_basic_credentials.setPassword(decoded_password); + } + else { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + } + + http_basic_credentials.authenticate(request); + } + auto session = makePooledHTTPSession(url, timeouts, 1); session->sendRequest(request); From 8e1de7897a0f950a44b9c67b5d7d97b47d380f25 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Wed, 24 Aug 2022 19:19:09 +0530 Subject: [PATCH 475/871] indentation fix --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index fe795608970..318ba3cb443 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -936,7 +936,8 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); - if (!url.getUserInfo().empty()) { + if (!url.getUserInfo().empty()) + { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; std::string decoded_username; @@ -944,14 +945,16 @@ private: http_credentials.fromUserInfo(url.getUserInfo()); - if (!http_credentials.getPassword().empty()) { + if (!http_credentials.getPassword().empty()) + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); Poco::URI::decode(http_credentials.getPassword(), decoded_password); http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else { + else + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); http_basic_credentials.setUsername(decoded_username); } From e3523cb1a463931513cb7f3edc9937d64ae82331 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 2 May 2023 12:31:00 +0000 Subject: [PATCH 476/871] test_for_basic_auth_registry - attempt to have two kafka instances --- .../runner/compose/docker_compose_kafka.yml | 44 +++++++++++-- tests/integration/helpers/cluster.py | 58 +++++++++++++++-- .../test_format_avro_confluent/test.py | 65 +++++++++++++++---- 3 files changed, 145 insertions(+), 22 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 39247f1dd37..fc476c09378 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,6 +31,28 @@ services: security_opt: - label:disable + kafka2: + image: confluentinc/cp-kafka:5.2.0 + hostname: kafka2 + ports: + - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + environment: + # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # KAFKA_HOST: ${KAFKA2_HOST} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + KAFKA_ADVERTISED_HOST_NAME: kafka2 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 2 + KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + depends_on: + - kafka_zookeeper + security_opt: + - label:disable + schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry @@ -39,12 +61,24 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + depends_on: + - kafka_zookeeper + - kafka1 + security_opt: + - label:disable + + schema-registry-auth: + image: confluentinc/cp-schema-registry:5.2.0 + hostname: schema-registry-auth + ports: + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + environment: + SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth + SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO - # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein - # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" @@ -52,6 +86,6 @@ services: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka1 + - kafka2 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9b5b33b1968..e0286f6e5c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,12 +481,18 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" + self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 + self._kafka2_port = 0 self.kafka_docker_id = None + self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 + self.schema_registry_auth_host = "schema-registry-auth" + self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -650,6 +656,13 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port + @property + def kafka2_port(self): + if self._kafka2_port: + return self._kafka2_port + self._kafka2_port = get_free_port() + return self._kafka2_port + @property def schema_registry_port(self): if self._schema_registry_port: @@ -657,6 +670,13 @@ class ClickHouseCluster: self._schema_registry_port = get_free_port() return self._schema_registry_port + @property + def schema_registry_auth_port(self): + if self._schema_registry_auth_port: + return self._schema_registry_auth_port + self._schema_registry_auth_port = get_free_port() + return self._schema_registry_auth_port + @property def kerberized_kafka_port(self): if self._kerberized_kafka_port: @@ -1164,10 +1184,13 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host + env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -2498,20 +2521,44 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - sr_client = CachedSchemaRegistryClient( - {"url": "http://localhost:{}".format(self.schema_registry_port)} - ) + reg_url="http://localhost:{}".format(self.schema_registry_port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) + start = time.time() + sr_started = False + sr_auth_started = False while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) logging.debug("Connected to SchemaRegistry") - return sr_client + sr_started = True + break except Exception as ex: logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) - raise Exception("Can't wait Schema Registry to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") + + + auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) + auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + + sr_auth_client = CachedSchemaRegistryClient(auth_arg) + while time.time() - start < timeout: + try: + sr_auth_client._send_request(sr_auth_client.url) + logging.debug("Connected to SchemaRegistry with auth") + sr_auth_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) + time.sleep(1) + + if not sr_auth_started: + raise Exception("Can't wait Schema Registry with auth to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2718,6 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) + self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 921fbdf5ef4..7261ce1b97d 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -37,15 +37,18 @@ def run_query(instance, query, data=None, settings=None): return result + + # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) + # arg={'url':reg_url} + # schema_registry_client = CachedSchemaRegistryClient(arg) + + def test_select(started_cluster): # type: (ClickHouseCluster) -> None - input("Cluster created, press any key to destroy...") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - schema_registry_client = CachedSchemaRegistryClient(arg) + schema_registry_client = CachedSchemaRegistryClient( + "http://localhost:{}".format(started_cluster.schema_registry_port) + ) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -65,12 +68,7 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - # schema_registry_url = "http://{}:{}@{}:{}".format( - # 'schemauser', 'letmein', - # started_cluster.schema_registry_host, 8081 - # ) - schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', + schema_registry_url = "http://{}:{}".format( started_cluster.schema_registry_host, 8081 ) @@ -83,3 +81,46 @@ def test_select(started_cluster): ["1"], ["2"], ] + + +# def test_select_auth(started_cluster): +# # type: (ClickHouseCluster) -> None + +# reg_url="http://localhost:{}".format( +# started_cluster.schema_registry_auth_port) +# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + +# schema_registry_client = CachedSchemaRegistryClient(arg) +# serializer = MessageSerializer(schema_registry_client) + +# schema = avro.schema.make_avsc_object( +# { +# "name": "test_record", +# "type": "record", +# "fields": [{"name": "value", "type": "long"}], +# } +# ) + +# buf = io.BytesIO() +# for x in range(0, 3): +# message = serializer.encode_record_with_schema( +# "test_subject", schema, {"value": x} +# ) +# buf.write(message) +# data = buf.getvalue() + +# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance +# schema_registry_url = "http://{}:{}@{}:{}".format( +# 'schemauser', 'letmein', +# started_cluster.schema_registry_auth_host, 8081 +# ) + +# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") +# settings = {"format_avro_schema_registry_url": schema_registry_url} +# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) +# stdout = run_query(instance, "select * from avro_data_auth") +# assert list(map(str.split, stdout.splitlines())) == [ +# ["0"], +# ["1"], +# ["2"], +# ] From fb3a860d7f02ddf321875eefefeeaeb46b265bf9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 10:56:35 +0000 Subject: [PATCH 477/871] test_for_basic_auth_registry - one kafka instance again --- .../runner/compose/docker_compose_kafka.yml | 49 ++++++++++--------- tests/integration/helpers/cluster.py | 2 +- 2 files changed, 27 insertions(+), 24 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index fc476c09378..e0b58fee73d 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,27 +31,27 @@ services: security_opt: - label:disable - kafka2: - image: confluentinc/cp-kafka:5.2.0 - hostname: kafka2 - ports: - - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - environment: - # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # KAFKA_HOST: ${KAFKA2_HOST} - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - KAFKA_ADVERTISED_HOST_NAME: kafka2 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 2 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - depends_on: - - kafka_zookeeper - security_opt: - - label:disable + # kafka2: + # image: confluentinc/cp-kafka:5.2.0 + # hostname: kafka2 + # ports: + # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + # environment: + # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # # KAFKA_HOST: ${KAFKA2_HOST} + # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + # KAFKA_ADVERTISED_HOST_NAME: kafka2 + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + # KAFKA_BROKER_ID: 2 + # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + # depends_on: + # - kafka_zookeeper + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -77,15 +77,18 @@ services: SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka2 + - kafka1 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e0286f6e5c9..e261364ab05 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2765,7 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) + # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: From db8e96147a9deb92364c8276577dedf68b7653a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 17:52:15 +0000 Subject: [PATCH 478/871] test_for_basic_auth_registry - started, but only auth test works --- .../runner/compose/docker_compose_kafka.yml | 94 +++++++++---------- .../test_format_avro_confluent/test.py | 27 ++++-- 2 files changed, 60 insertions(+), 61 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index e0b58fee73d..47e41812cf5 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,90 +1,80 @@ version: '2.3' services: + # kafka_zookeeper: + # image: zookeeper:3.4.9 + # hostname: kafka_zookeeper + # environment: + # ZOO_MY_ID: 1 + # ZOO_PORT: 2181 + # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 + # security_opt: + # - label:disable kafka_zookeeper: - image: zookeeper:3.4.9 - hostname: kafka_zookeeper + image: confluentinc/cp-zookeeper + ports: + - 2181:2181 environment: - ZOO_MY_ID: 1 - ZOO_PORT: 2181 - ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - security_opt: - - label:disable - + ZOOKEEPER_CLIENT_PORT: 2181 + # security_opt: + # - label:disable kafka1: - image: confluentinc/cp-kafka:5.2.0 + image: confluentinc/cp-kafka hostname: kafka1 ports: - - ${KAFKA_EXTERNAL_PORT:-8081}:${KAFKA_EXTERNAL_PORT:-8081} + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + # KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - security_opt: - - label:disable - - # kafka2: - # image: confluentinc/cp-kafka:5.2.0 - # hostname: kafka2 - # ports: - # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - # environment: - # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # # KAFKA_HOST: ${KAFKA2_HOST} - # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - # KAFKA_ADVERTISED_HOST_NAME: kafka2 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - # KAFKA_BROKER_ID: 2 - # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - # depends_on: - # - kafka_zookeeper - # security_opt: - # - label:disable + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - ${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} depends_on: - kafka_zookeeper - kafka1 - security_opt: - - label:disable + # security_opt: + # - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry-auth ports: - - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth + # SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 7261ce1b97d..cd0906bedee 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -1,5 +1,6 @@ import io import logging +import time import avro.schema import pytest @@ -46,14 +47,21 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + time.sleep(3) + + # schema_registry_client = CachedSchemaRegistryClient( + # "http://localhost:{}".format(started_cluster.schema_registry_port) + # ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( { - "name": "test_record", + "name": "test_record1", "type": "record", "fields": [{"name": "value", "type": "long"}], } @@ -62,14 +70,14 @@ def test_select(started_cluster): buf = io.BytesIO() for x in range(0, 3): message = serializer.encode_record_with_schema( - "test_subject", schema, {"value": x} + "test_subject1", schema, {"value": x} ) buf.write(message) data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - started_cluster.schema_registry_host, 8081 + started_cluster.schema_registry_host, started_cluster.schema_registry_port ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") @@ -85,6 +93,7 @@ def test_select(started_cluster): # def test_select_auth(started_cluster): # # type: (ClickHouseCluster) -> None +# time.sleep(5) # reg_url="http://localhost:{}".format( # started_cluster.schema_registry_auth_port) @@ -95,7 +104,7 @@ def test_select(started_cluster): # schema = avro.schema.make_avsc_object( # { -# "name": "test_record", +# "name": "test_record_auth", # "type": "record", # "fields": [{"name": "value", "type": "long"}], # } @@ -104,7 +113,7 @@ def test_select(started_cluster): # buf = io.BytesIO() # for x in range(0, 3): # message = serializer.encode_record_with_schema( -# "test_subject", schema, {"value": x} +# "test_subject_auth", schema, {"value": x} # ) # buf.write(message) # data = buf.getvalue() @@ -112,7 +121,7 @@ def test_select(started_cluster): # instance = started_cluster.instances["dummy"] # type: ClickHouseInstance # schema_registry_url = "http://{}:{}@{}:{}".format( # 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, 8081 +# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port # ) # run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") From 66581d091af3eda08591e12af551e83a88a95520 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 07:08:04 +0000 Subject: [PATCH 479/871] test_for_basic_auth_registry - both tests works, simplifications --- .../runner/compose/docker_compose_kafka.yml | 46 +++---- tests/integration/helpers/cluster.py | 70 +++------- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 122 ++++++++++++------ 4 files changed, 116 insertions(+), 123 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 47e41812cf5..5e2e9d87c39 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,43 +1,33 @@ version: '2.3' services: - # kafka_zookeeper: - # image: zookeeper:3.4.9 - # hostname: kafka_zookeeper - # environment: - # ZOO_MY_ID: 1 - # ZOO_PORT: 2181 - # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - # security_opt: - # - label:disable kafka_zookeeper: - image: confluentinc/cp-zookeeper + image: zookeeper:3.4.9 + hostname: kafka_zookeeper ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 - # security_opt: - # - label:disable + ZOOKEEPER_CLIENT_PORT: 2181 + security_opt: + - label:disable + kafka1: - image: confluentinc/cp-kafka + image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT - # KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -46,15 +36,14 @@ services: - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 - SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT} + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: noauth depends_on: - kafka_zookeeper - kafka1 - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 @@ -62,19 +51,14 @@ services: ports: - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - # SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: auth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e261364ab05..c51c97ee6c4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,18 +481,14 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" - self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 - self._kafka2_port = 0 self.kafka_docker_id = None - self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 self.schema_registry_auth_host = "schema-registry-auth" self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) - self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -656,13 +652,6 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port - @property - def kafka2_port(self): - if self._kafka2_port: - return self._kafka2_port - self._kafka2_port = get_free_port() - return self._kafka2_port - @property def schema_registry_port(self): if self._schema_registry_port: @@ -1184,12 +1173,9 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host - env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) - env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] @@ -2521,44 +2507,27 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - reg_url="http://localhost:{}".format(self.schema_registry_port) - arg={'url':reg_url} - sr_client = CachedSchemaRegistryClient(arg) + for port in self.schema_registry_port, self.schema_registry_auth_port: + reg_url="http://localhost:{}".format(port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) - start = time.time() - sr_started = False - sr_auth_started = False - while time.time() - start < timeout: - try: - sr_client._send_request(sr_client.url) - logging.debug("Connected to SchemaRegistry") - sr_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) - time.sleep(1) + start = time.time() + sr_started = False + sr_auth_started = False + while time.time() - start < timeout: + try: + sr_client._send_request(sr_client.url) + logging.debug("Connected to SchemaRegistry") + # don't care about possible auth errors + sr_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) + time.sleep(1) - if not sr_started: - raise Exception("Can't wait Schema Registry to start") - - - auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) - auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - - sr_auth_client = CachedSchemaRegistryClient(auth_arg) - while time.time() - start < timeout: - try: - sr_auth_client._send_request(sr_auth_client.url) - logging.debug("Connected to SchemaRegistry with auth") - sr_auth_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) - time.sleep(1) - - if not sr_auth_started: - raise Exception("Can't wait Schema Registry with auth to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2765,7 +2734,6 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 8903cf6edd6..7fde510bf5a 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1 +1,2 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cd0906bedee..d58f6d972d1 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -9,7 +9,7 @@ from confluent_kafka.avro.cached_schema_registry_client import ( ) from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance - +from urllib import parse @pytest.fixture(scope="module") def started_cluster(): @@ -47,11 +47,8 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(3) + # input("Top of test_select, press any key") - # schema_registry_client = CachedSchemaRegistryClient( - # "http://localhost:{}".format(started_cluster.schema_registry_port) - # ) reg_url="http://localhost:{}".format( started_cluster.schema_registry_port) arg={'url':reg_url} @@ -91,45 +88,88 @@ def test_select(started_cluster): ] -# def test_select_auth(started_cluster): -# # type: (ClickHouseCluster) -> None -# time.sleep(5) +def test_select_auth(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) -# reg_url="http://localhost:{}".format( -# started_cluster.schema_registry_auth_port) -# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} -# schema_registry_client = CachedSchemaRegistryClient(arg) -# serializer = MessageSerializer(schema_registry_client) + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) -# schema = avro.schema.make_avsc_object( -# { -# "name": "test_record_auth", -# "type": "record", -# "fields": [{"name": "value", "type": "long"}], -# } -# ) + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) -# buf = io.BytesIO() -# for x in range(0, 3): -# message = serializer.encode_record_with_schema( -# "test_subject_auth", schema, {"value": x} -# ) -# buf.write(message) -# data = buf.getvalue() + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() -# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance -# schema_registry_url = "http://{}:{}@{}:{}".format( -# 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port -# ) + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) -# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") -# settings = {"format_avro_schema_registry_url": schema_registry_url} -# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) -# stdout = run_query(instance, "select * from avro_data_auth") -# assert list(map(str.split, stdout.splitlines())) == [ -# ["0"], -# ["1"], -# ["2"], -# ] + run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] + +def test_select_auth_encoded(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) + + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) + + run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth_encoded") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 83569688cba16b80ed959c054fd2f36187c520d4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 12:59:08 +0000 Subject: [PATCH 480/871] test_for_basic_auth_registry - UnknownTopicOrPartitionException --- docker/test/integration/runner/compose/docker_compose_kafka.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 5e2e9d87c39..d701af1d425 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -42,6 +42,7 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable @@ -64,5 +65,6 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable From c1c5ffa309c20899f81548bd3314233d84eb03e1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 13:29:19 +0000 Subject: [PATCH 481/871] test_for_basic_auth_registry - cpp code small improvement --- .../Formats/Impl/AvroRowInputFormat.cpp | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 318ba3cb443..a7efc823fbb 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -940,24 +940,19 @@ private: { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; - std::string decoded_username; - std::string decoded_password; http_credentials.fromUserInfo(url.getUserInfo()); + std::string decoded_username; + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + if (!http_credentials.getPassword().empty()) { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); + std::string decoded_password; Poco::URI::decode(http_credentials.getPassword(), decoded_password); - - http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else - { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); - http_basic_credentials.setUsername(decoded_username); - } http_basic_credentials.authenticate(request); } From c550a532e649bfde1382bc7f56cac6a38dee0dee Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 20:49:43 +0000 Subject: [PATCH 482/871] test_for_basic_auth_registry - black formatter happy + some doc --- .../operations/settings/settings-formats.md | 11 ++++ tests/integration/helpers/cluster.py | 21 ++++---- .../test_format_avro_confluent/test.py | 54 ++++++++++++------- 3 files changed, 59 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..637ade17296 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1325,6 +1325,17 @@ Default value: 0. Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format. +Format: +``` text +http://[user:password@]machine[:port]" +``` + +Examples: +``` text +http://registry.example.com:8081 +http://admin:secret@registry.example.com:8081 +``` + Default value: `Empty`. ### output_format_avro_codec {#output_format_avro_codec} diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c51c97ee6c4..c52442ecb9c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1168,15 +1168,15 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd( - self, instance, env_variables, docker_compose_yml_dir - ): + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str( + self.schema_registry_auth_port + ) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -1617,7 +1617,10 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, - with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, + with_secrets=with_secrets + or with_kerberized_hdfs + or with_kerberos_kdc + or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -2508,8 +2511,8 @@ class ClickHouseCluster: def wait_schema_registry_to_start(self, timeout=180): for port in self.schema_registry_port, self.schema_registry_auth_port: - reg_url="http://localhost:{}".format(port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(port) + arg = {"url": reg_url} sr_client = CachedSchemaRegistryClient(arg) start = time.time() @@ -4245,8 +4248,8 @@ class ClickHouseInstance: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path - from_dir=self.secrets_dir - to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + from_dir = self.secrets_dir + to_dir = p.abspath(p.join(base_secrets_dir, "secrets")) logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( self.secrets_dir, diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index d58f6d972d1..61d839ee63e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -11,6 +11,7 @@ from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance from urllib import parse + @pytest.fixture(scope="module") def started_cluster(): try: @@ -37,8 +38,6 @@ def run_query(instance, query, data=None, settings=None): return result - - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) # arg={'url':reg_url} # schema_registry_client = CachedSchemaRegistryClient(arg) @@ -49,9 +48,8 @@ def test_select(started_cluster): # input("Top of test_select, press any key") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) + arg = {"url": reg_url} schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -92,9 +90,12 @@ def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -117,13 +118,17 @@ def test_select_auth(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + "schemauser", + "letmein", + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + run_query( + instance, "insert into avro_data_auth format AvroConfluent", data, settings + ) stdout = run_query(instance, "select * from avro_data_auth") assert list(map(str.split, stdout.splitlines())) == [ ["0"], @@ -131,13 +136,17 @@ def test_select_auth(started_cluster): ["2"], ] + def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -160,13 +169,22 @@ def test_select_auth_encoded(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + parse.quote_plus("schemauser/slash"), + parse.quote_plus("letmein"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) - run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + run_query( + instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()" + ) settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + run_query( + instance, + "insert into avro_data_auth_encoded format AvroConfluent", + data, + settings, + ) stdout = run_query(instance, "select * from avro_data_auth_encoded") assert list(map(str.split, stdout.splitlines())) == [ ["0"], From aa2b7e248d719ed94386e5cf066ed03fe71fef12 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 08:37:46 +0000 Subject: [PATCH 483/871] test_for_basic_auth_registry - fix port in test_kafka_formats --- .../runner/compose/docker_compose_kafka.yml | 7 ++++--- tests/integration/test_storage_kafka/test.py | 14 +++++++------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index d701af1d425..c0185afb7df 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -17,10 +17,11 @@ services: ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9a6d3e0513c..d0686c7c36f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -762,7 +762,7 @@ def test_kafka_formats(kafka_cluster): ), ], "extra_settings": ", format_avro_schema_registry_url='http://{}:{}'".format( - kafka_cluster.schema_registry_host, 8081 + kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port ), "supports_empty_value": True, }, @@ -4339,7 +4339,7 @@ def test_row_based_formats(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -4347,10 +4347,10 @@ def test_row_based_formats(kafka_cluster): kafka_group_name = '{format_name}', kafka_format = '{format_name}', kafka_max_rows_per_message = 5; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); """ ) @@ -4459,17 +4459,17 @@ def test_block_based_formats_2(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{format_name}', kafka_group_name = '{format_name}', kafka_format = '{format_name}'; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) From 4259176f24b223decafd0d07bef430a30844e850 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 09:23:28 +0000 Subject: [PATCH 484/871] test_for_basic_auth_registry - original zk configuration restored --- .../test/integration/runner/compose/docker_compose_kafka.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index c0185afb7df..30d1b0bed3f 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -7,7 +7,9 @@ services: ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 + ZOO_MY_ID: 1 + ZOO_PORT: 2181 + ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 security_opt: - label:disable From f1ce1da00744f17c42d94f1736417474eba478fe Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 18 May 2023 23:21:29 +0000 Subject: [PATCH 485/871] test_for_basic_auth_registry - new test and cleanup per code review --- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 56 ++++++++++++++++++- 2 files changed, 55 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 7fde510bf5a..a367925c806 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1,2 +1,3 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +complexschemauser: MD5:fcaeda86837fcd37755044e7258edc5d,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 61d839ee63e..2d78668f000 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -88,7 +88,6 @@ def test_select(started_cluster): def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -139,7 +138,6 @@ def test_select_auth(started_cluster): def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -191,3 +189,57 @@ def test_select_auth_encoded(started_cluster): ["1"], ["2"], ] + +def test_select_auth_encoded_complex(started_cluster): + # type: (ClickHouseCluster) -> None + + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded_complex", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded_complex", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus("complexschemauser"), + parse.quote_plus("letmein%@:/"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, + ) + + run_query( + instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + ) + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query( + instance, + "insert into avro_data_auth_encoded_complex format AvroConfluent", + data, + settings, + ) + stdout = run_query(instance, "select * from avro_data_auth_encoded_complex") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 9f6ab5e816378dce815957e396cf4389986256e4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 19 May 2023 09:31:24 +0000 Subject: [PATCH 486/871] test_for_basic_auth_registry - made black formatter happy --- tests/integration/test_format_avro_confluent/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 2d78668f000..b27642c921b 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -190,6 +190,7 @@ def test_select_auth_encoded(started_cluster): ["2"], ] + def test_select_auth_encoded_complex(started_cluster): # type: (ClickHouseCluster) -> None @@ -228,7 +229,8 @@ def test_select_auth_encoded_complex(started_cluster): ) run_query( - instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + instance, + "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()", ) settings = {"format_avro_schema_registry_url": schema_registry_url} run_query( From c8347bd31300bec4cdd3277680f398808d37533c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 15:55:44 +0000 Subject: [PATCH 487/871] test_for_basic_auth_registry: some comments removed per code review --- tests/integration/test_format_avro_confluent/test.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index b27642c921b..540f90ae05e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -38,16 +38,10 @@ def run_query(instance, query, data=None, settings=None): return result - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) - # arg={'url':reg_url} - # schema_registry_client = CachedSchemaRegistryClient(arg) - def test_select(started_cluster): # type: (ClickHouseCluster) -> None - # input("Top of test_select, press any key") - reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) arg = {"url": reg_url} From 2d46052d62bf8b7efd16aeb769e278b9df54971b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:35:46 +0000 Subject: [PATCH 488/871] 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 0838e0366df..75d1e493873 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 aa888ad64a95ef801977844b2b253bb8162cfc1a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 08:46:57 +0000 Subject: [PATCH 489/871] Separate thread mutex, add test --- src/Common/SystemLogBase.cpp | 2 +- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/SystemLog.cpp | 2 +- src/Interpreters/SystemLog.h | 1 + tests/queries/0_stateless/02813_starting_in_text_log.reference | 1 + tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 ++ 6 files changed, 8 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02813_starting_in_text_log.reference create mode 100755 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index baee7021c35..bed6d661db7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -214,7 +214,7 @@ SystemLogBase::SystemLogBase( template void SystemLogBase::startup() { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); saving_thread = std::make_unique([this] { savingThreadFunction(); }); } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index fa9f9b6f72e..0ac376769ad 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -67,6 +67,7 @@ public: virtual void savingThreadFunction() = 0; protected: + std::mutex thread_mutex; std::unique_ptr saving_thread; bool is_shutdown = false; @@ -93,10 +94,10 @@ public: Index pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread); void confirm(Index to_flush_end); +private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; -private: Poco::Logger * log; // Queue is bounded. But its size is quite large to not block in all normal cases. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 674210cbaad..0b89b1dec26 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -358,7 +358,7 @@ template void SystemLog::stopFlushThread() { { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); if (!saving_thread || !saving_thread->joinable()) return; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 91fb7f49221..5d8bb30150d 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -124,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using ISystemLog::thread_mutex; using Base::queue; private: diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.reference b/tests/queries/0_stateless/02813_starting_in_text_log.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql new file mode 100755 index 00000000000..8ef78945a72 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -0,0 +1,2 @@ +SYSTEM FLUSH LOGS; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; From 70543e8ef9fe8523c5604d62fac3376da91c6d2c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 08:47:53 +0000 Subject: [PATCH 490/871] Automatic style fix --- .../test_replicated_database/test.py | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 17dd2adcde4..ed034a326da 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -302,12 +302,21 @@ def test_alter_attach(started_cluster, attachable_part, engine): ) main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) else: - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "" + ) main_node.query(f"DROP DATABASE {database} SYNC") dummy_node.query(f"DROP DATABASE {database} SYNC") @@ -333,7 +342,9 @@ def test_alter_drop_part(started_cluster, engine): assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + ) else: assert ( dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") From 96f048f7f8895507c6827f373699244f345730ec Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 19 Jul 2023 10:46:02 +0200 Subject: [PATCH 491/871] Convert output UInt128 to FixedString even if input is empty --- src/Functions/FunctionsHashing.h | 48 +++++++++---------- .../0_stateless/02534_keyed_siphash.reference | 2 + .../0_stateless/02534_keyed_siphash.sql | 3 ++ 3 files changed, 29 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 4965d1f7b49..82944630b10 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1535,33 +1535,33 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count == 0) - return col_to; - - typename ColumnVector::Container & vec_to = col_to->getData(); - - /// If using a "keyed" algorithm, the first argument is the key and - /// the data starts from the second argument. - /// Otherwise there is no key and all arguments are interpreted as data. - constexpr size_t first_data_argument = Keyed; - - if (arguments.size() <= first_data_argument) + if (input_rows_count != 0) { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + typename ColumnVector::Container & vec_to = col_to->getData(); - KeyColumnsType key_cols{}; - if constexpr (Keyed) - if (!arguments.empty()) - key_cols = Impl::parseKeyColumns(arguments[0]); + /// If using a "keyed" algorithm, the first argument is the key and + /// the data starts from the second argument. + /// Otherwise there is no key and all arguments are interpreted as data. + constexpr size_t first_data_argument = Keyed; - /// The function supports arbitrary number of arguments of arbitrary types. - bool is_first_argument = true; - for (size_t i = first_data_argument; i < arguments.size(); ++i) - { - const auto & col = arguments[i]; - executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + if (arguments.size() <= first_data_argument) + { + /// Return a fixed random-looking magic number when input is empty + vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); + } + + KeyColumnsType key_cols{}; + if constexpr (Keyed) + if (!arguments.empty()) + key_cols = Impl::parseKeyColumns(arguments[0]); + + /// The function supports arbitrary number of arguments of arbitrary types. + bool is_first_argument = true; + for (size_t i = first_data_argument; i < arguments.size(); ++i) + { + const auto & col = arguments[i]; + executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + } } if constexpr (std::is_same_v) /// backward-compatible diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index de783d7dddf..a9f724365a8 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -232,3 +232,5 @@ Check multiple keys as separate ints from a table with constant data 9357996107237883963 86AE90BB6A238D3F6221457630142C9B F6D93D8FEA6D7DECCDD95A7A0A2AA36D +Check asan bug +0 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 14b422ac713..4f3ae7d62bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -331,3 +331,6 @@ INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; + +SELECT 'Check asan bug'; +SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); From f22452c78c36c2d2529444a137d4853749c04945 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:54:31 +0000 Subject: [PATCH 492/871] Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Output of SYSTEM.QUERY_CACHE - before this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS [...] - after this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS use_query_cache = 1 [...] --- src/Interpreters/Cache/QueryCache.cpp | 35 ++++++++++--------- src/Interpreters/Cache/QueryCache.h | 8 +++-- .../System/StorageSystemQueryCache.cpp | 2 +- .../02494_query_cache_secrets.reference | 2 +- 4 files changed, 26 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index f46a10ca51d..1d1543844a2 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -115,6 +115,15 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } +String queryStringFromAst(ASTPtr ast) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings format_settings(buf, /*one_line*/ true); + format_settings.show_secrets = false; + ast->format(format_settings); + return buf.str(); +} + } QueryCache::Key::Key( @@ -129,6 +138,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) + , query_string(queryStringFromAst(ast_)) { } @@ -142,15 +152,6 @@ bool QueryCache::Key::operator==(const Key & other) const return ast->getTreeHash() == other.ast->getTreeHash(); } -String QueryCache::Key::queryStringFromAst() const -{ - WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); - return buf.str(); -} - size_t QueryCache::KeyHasher::operator()(const Key & key) const { SipHash hash; @@ -191,7 +192,7 @@ QueryCache::Writer::Writer( if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { skip_insert = true; /// Key already contained in cache and did not expire yet --> don't replace it - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); } } @@ -263,14 +264,14 @@ void QueryCache::Writer::finalizeWrite() if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.query_string); return; } if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { /// Same check as in ctor because a parallel Writer could have inserted the current key in the meantime - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); return; } @@ -353,7 +354,7 @@ void QueryCache::Writer::finalizeWrite() if ((new_entry_size_in_bytes > max_entry_size_in_bytes) || (new_entry_size_in_rows > max_entry_size_in_rows)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.query_string); return; } @@ -388,7 +389,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry.has_value()) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.query_string); return; } @@ -397,13 +398,13 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry_key.is_shared && entry_key.user_name != key.user_name) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.query_string); return; } if (IsStale()(entry_key)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.query_string); return; } @@ -441,7 +442,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes); } - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.query_string); } bool QueryCache::Reader::hasCacheEntryForKey() const diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 6ef7cc60918..a67adcc86c9 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -30,7 +30,7 @@ public: /// ---------------------------------------------------- /// The actual key (data which gets hashed): - /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select) + /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select). const ASTPtr ast; /// Note: For a transactionally consistent cache, we would need to include the system settings in the cache key or invalidate the @@ -58,6 +58,11 @@ public: /// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case) const bool is_compressed; + /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the + /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in + /// the SYSTEM.QUERY_CACHE. + const String query_string; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, @@ -69,7 +74,6 @@ public: Key(ASTPtr ast_, const String & user_name_); bool operator==(const Key & other) const; - String queryStringFromAst() const; }; struct Entry diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..288e4fd52a0 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -44,7 +44,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr if (!key.is_shared && key.user_name != user_name) continue; - res_columns[0]->insert(key.queryStringFromAst()); /// approximates the original query string + res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); res_columns[3]->insert(key.is_shared); diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.reference b/tests/queries/0_stateless/02494_query_cache_secrets.reference index dd6341262bc..306374eed4b 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.reference +++ b/tests/queries/0_stateless/02494_query_cache_secrets.reference @@ -1,2 +1,2 @@ A2193552DCF8A9F99AC35F86BC4D2FFD -SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS +SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS use_query_cache = 1 From d0e886070c6acbb57c35a1d3df287e2a07a9f115 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 17:39:00 +0800 Subject: [PATCH 493/871] system.events and system.metrics tables add column name as an alias to event and metric --- src/Storages/System/StorageSystemEvents.cpp | 7 +++++++ src/Storages/System/StorageSystemEvents.h | 1 + src/Storages/System/StorageSystemMetrics.cpp | 7 +++++++ src/Storages/System/StorageSystemMetrics.h | 1 + 4 files changed, 16 insertions(+) diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index b9b07cfe0ac..a914c60abf4 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -16,6 +16,13 @@ NamesAndTypesList StorageSystemEvents::getNamesAndTypes() }; } +NamesAndAliases StorageSystemEvents::getNamesAndAliases() +{ + return { + {"name", std::make_shared(), "event"} + }; +} + void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index ea0322c13b8..ed80a441529 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -17,6 +17,7 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemMetrics.cpp b/src/Storages/System/StorageSystemMetrics.cpp index 6007c8a7c71..ced363fed17 100644 --- a/src/Storages/System/StorageSystemMetrics.cpp +++ b/src/Storages/System/StorageSystemMetrics.cpp @@ -17,6 +17,13 @@ NamesAndTypesList StorageSystemMetrics::getNamesAndTypes() }; } +NamesAndAliases StorageSystemMetrics::getNamesAndAliases() +{ + return { + {"name", std::make_shared(), "metric"} + }; +} + void StorageSystemMetrics::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index def95e0a934..bffc7e6f1fc 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -18,6 +18,7 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; From d3663c356f4a8cab8b77f36fcc9b8251e6ddd02e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 12:11:57 +0200 Subject: [PATCH 494/871] Wait fo KILL MUTATION to finish --- .../00834_kill_mutation_replicated_zookeeper.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 2e917f67fe8..16ad08deeb2 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -57,6 +57,14 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM system.mutations WHERE database ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" +# Wait for the 1st mutation to be actually killed and the 2nd to finish +query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) +while [ "$query_result" != "0" ] +do + query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) + sleep 0.5 +done + ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r2" From 95424177d5de5bd7973823ffdaaacafce442e8ba Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 18:26:54 +0800 Subject: [PATCH 495/871] review fix --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 12 +-- src/Core/Settings.h | 3 +- src/Formats/FormatFactory.cpp | 3 +- src/Formats/FormatSettings.h | 3 +- .../Formats/Impl/CSVRowInputFormat.cpp | 78 ++++++++++--------- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 8 +- .../02811_csv_input_field_type_mismatch.sh | 4 +- .../data_csv/csv_with_bad_field_values.csv | 5 ++ .../data_csv/csv_with_diff_field_types.csv | 3 - 11 files changed, 62 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f45c55a9734..c20f304c346 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_field_deserialization](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_field_deserialization) - Allow to check whether the csv input field can be successful deserialized. Default value - `false`. -- [input_format_csv_set_default_if_deserialization_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialization_failed) - Set default value to column if the csv input field deserialization failed. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 409ac4bd58a..5fac8df02d7 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,15 +969,9 @@ Result a b ``` -### input_format_csv_allow_check_field_deserialization {#input_format_csv_allow_check_field_deserialization} +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to use whitespace or tab as field delimiter in CSV strings. - -Default value: `false`. - -### input_format_csv_set_default_if_deserialization_failed {#input_format_csv_set_default_if_deserialization_failed} - -Allow to set default value to column if the csv input field's deserialization failed +Allow to set default value to column when CSV field deserialize failed on bad value Default value: `false`. @@ -988,7 +982,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_field_deserialization=true, input_format_csv_set_default_if_deserialization_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d93ba9ad2c..311813fb38f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_field_deserialization, false, "Allow to check the csv input field deserialization whether success or not.", 0) \ - M(Bool, input_format_csv_set_default_if_deserialization_failed, false, "All to set column default value if the input field's deserialization failed.", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b3b9609f9fe..3df2ca7d2e0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,8 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.allow_check_field_deserialization = settings.input_format_csv_allow_check_field_deserialization; - format_settings.csv.set_default_if_deserialization_failed = settings.input_format_csv_set_default_if_deserialization_failed; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 4d4eb926992..4e49d338e43 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_field_deserialization=false; - bool set_default_if_deserialization_failed=false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index e1be6b21610..34d5b589591 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -316,49 +317,52 @@ bool CSVFormatReader::readField( return false; } - BufferBase::Position pos_start = buf->position(); + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + { + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + } + + /// Read the column normally. + serialization->deserializeTextCSV(column, istr, format_settings); + return true; +} + +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_field_deserialization) - { - std::string field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromMemory tmp(field); - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); - else - serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && !tmp.eof()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); - } - else - { - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - { - /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); - } - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); - } + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; } - catch (Exception & e) + catch (const Exception &) { - LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.set_default_if_deserialization_failed) - { - // Reset the column and buffer position, then skip the field and set column default value. - if (column.size() == col_size + 1) - column.popBack(1); - buf->position() = pos_start; - skipField(); - column.insertDefault(); - } - else - throw; + is_bad_value = true; } - return true; + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; } void CSVFormatReader::skipPrefixBeforeHeader() diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index c5ee611a230..19c7956ba84 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,3 +1,5 @@ -a 1 2023-03-14 -a 0 1970-01-01 -c 1 1970-01-01 +0 111 1970-01-01 2023-03-24 00:00:00 false +1 abc 2023-03-14 2023-03-14 11:22:33 true +2 c 1970-01-01 1970-01-01 08:00:00 false +4 888 2023-03-14 1970-06-03 14:43:53 false +5 bks 1970-01-01 2023-07-19 18:17:59 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index df736ea6792..3961664b9b3 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..faedd9b6705 --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,2023-03-14 11:22:33,true +2,c,ab,2023,false +bc,111,ab,2023-03-24,ban +4,888,2023-03-14,13243433,false +5,bks,2023-03,1689761879,abdd \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv deleted file mode 100644 index 464172c515c..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv +++ /dev/null @@ -1,3 +0,0 @@ -a,1,2023-03-14 -a,b,c -c,1,a \ No newline at end of file From 380b4ffe2be4107ae3965cba19c5b697e7108128 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 12:29:39 +0200 Subject: [PATCH 496/871] Reduce dependencies for skim by avoid using default features By default skim requires cli -> clap -> termcolor -> winapi-util Signed-off-by: Azat Khuzhin --- rust/skim/Cargo.lock | 204 ++++++++++++------------------------------- rust/skim/Cargo.toml | 2 +- 2 files changed, 58 insertions(+), 148 deletions(-) diff --git a/rust/skim/Cargo.lock b/rust/skim/Cargo.lock index 9f948ee1c38..f55ea8a84b0 100644 --- a/rust/skim/Cargo.lock +++ b/rust/skim/Cargo.lock @@ -42,17 +42,6 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -104,31 +93,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "clap" -version = "3.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ea181bf566f71cb9a5d17a59e1871af638180a18fb0035c92ae62b705207123" -dependencies = [ - "atty", - "bitflags", - "clap_lex", - "indexmap", - "once_cell", - "strsim", - "termcolor", - "textwrap", -] - -[[package]] -name = "clap_lex" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2850f2f5a82cbf437dd5af4d49848fbdfc27c157c3d010345776f952765261c5" -dependencies = [ - "os_str_bytes", -] - [[package]] name = "codespan-reporting" version = "0.11.1" @@ -214,9 +178,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88abab2f5abbe4c56e8f1fb431b784d710b709888f35755a160e62e33fe38e8" +checksum = "5032837c1384de3708043de9d4e97bb91290faca6c16529a28aa340592a78166" dependencies = [ "cc", "cxxbridge-flags", @@ -226,9 +190,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c0c11acd0e63bae27dcd2afced407063312771212b7a823b4fd72d633be30fb" +checksum = "51368b3d0dbf356e10fcbfd455a038503a105ee556f7ee79b6bb8c53a7247456" dependencies = [ "cc", "codespan-reporting", @@ -236,24 +200,24 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] name = "cxxbridge-flags" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3816ed957c008ccd4728485511e3d9aaf7db419aa321e3d2c5a2f3411e36c8" +checksum = "0d9062157072e4aafc8e56ceaf8325ce850c5ae37578c852a0d4de2cecdded13" [[package]] name = "cxxbridge-macro" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a26acccf6f445af85ea056362561a24ef56cdc15fcc685f03aec50b9c702cb6d" +checksum = "cf01e8a540f5a4e0f284595834f81cf88572f244b768f051724537afa99a2545" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -359,19 +323,6 @@ version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" -[[package]] -name = "env_logger" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12e6657c4c97ebab115a42dcee77225f7f482cdd841cf7088c657a42e9e00e7" -dependencies = [ - "atty", - "humantime", - "log", - "regex", - "termcolor", -] - [[package]] name = "fnv" version = "1.0.7" @@ -398,32 +349,11 @@ dependencies = [ "wasi 0.11.0+wasi-snapshot-preview1", ] -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" - [[package]] name = "hermit-abi" -version = "0.1.19" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" [[package]] name = "iana-time-zone" @@ -454,16 +384,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" -[[package]] -name = "indexmap" -version = "1.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" -dependencies = [ - "autocfg", - "hashbrown", -] - [[package]] name = "js-sys" version = "0.3.64" @@ -487,9 +407,9 @@ checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" [[package]] name = "link-cplusplus" -version = "1.0.8" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" +checksum = "9d240c6f7e1ba3a28b0249f774e6a9dd0175054b52dfbb61b16eb8505c3785c9" dependencies = [ "cc", ] @@ -564,7 +484,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.3.1", + "hermit-abi", "libc", ] @@ -574,12 +494,6 @@ version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" -[[package]] -name = "os_str_bytes" -version = "6.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" - [[package]] name = "pin-utils" version = "0.1.0" @@ -588,18 +502,18 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "proc-macro2" -version = "1.0.63" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.29" +version = "1.0.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +checksum = "5fe8a65d69dd0808184ebb5f836ab526bb259db23c657efa38711b1072ee47f0" dependencies = [ "proc-macro2", ] @@ -648,9 +562,21 @@ dependencies = [ [[package]] name = "regex" -version = "1.8.4" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0ab3ca65655bb1e41f2a8c8cd662eb4fb035e67c3f78da1d61dffe89d07300f" +checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" dependencies = [ "aho-corasick", "memchr", @@ -659,39 +585,33 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.2" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" +checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" [[package]] name = "rustversion" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "scopeguard" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "scratch" -version = "1.0.5" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" +checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" [[package]] name = "serde" -version = "1.0.164" +version = "1.0.171" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e8c8cf938e98f769bc164923b06dce91cea1751522f46f8466461af04c9027d" - -[[package]] -name = "shlex" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43b2853a4d09f215c24cc5489c992ce46052d359b5109343cbafbf26bc62f8a3" +checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" [[package]] name = "skim" @@ -699,23 +619,19 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ - "atty", "beef", "bitflags", "chrono", - "clap", "crossbeam", "defer-drop", "derive_builder", - "env_logger", "fuzzy-matcher", "lazy_static", "log", "nix 0.25.1", "rayon", "regex", - "shlex", - "time 0.3.22", + "time 0.3.23", "timer", "tuikit", "unicode-width", @@ -741,9 +657,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.23" +version = "2.0.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +checksum = "45c3457aacde3c65315de5031ec191ce46604304d2446e803d71ade03308d970" dependencies = [ "proc-macro2", "quote", @@ -770,30 +686,24 @@ dependencies = [ "winapi-util", ] -[[package]] -name = "textwrap" -version = "0.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" - [[package]] name = "thiserror" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" +checksum = "a35fc5b8971143ca348fa6df4f024d4d55264f3468c71ad1c2f365b0a4d58c42" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" +checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -819,9 +729,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.22" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" +checksum = "59e399c068f43a5d116fedaf73b203fa4f9c519f17e2b34f63221d3792f81446" dependencies = [ "serde", "time-core", @@ -858,9 +768,9 @@ dependencies = [ [[package]] name = "unicode-ident" -version = "1.0.9" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15811caf2415fb889178633e7724bad2509101cde276048e013b9def5e51fa0" +checksum = "301abaae475aa91687eb82514b328ab47a211a533026cb25fc3e519b86adfc3c" [[package]] name = "unicode-width" @@ -928,7 +838,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-shared", ] @@ -950,7 +860,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index e5801a26f77..0381ad81619 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -6,7 +6,7 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -skim = "0.10.2" +skim = { version = "0.10.2", default-features = false } cxx = "1.0.83" term = "0.7.0" From af6361e2a0c78f45500a37bc67f563bd74412076 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 19 Jul 2023 12:35:52 +0200 Subject: [PATCH 497/871] Fix 02725_memory-for-merges --- tests/queries/0_stateless/02725_memory-for-merges.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index 347c8b2a8d3..1a8402dff4b 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-random-merge-tree-settings -- We allocate a lot of memory for buffers when reading or writing to S3 DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; @@ -21,7 +21,6 @@ OPTIMIZE TABLE 02725_memory_for_merges FINAL; SYSTEM FLUSH LOGS; -WITH (SELECT uuid FROM system.tables WHERE table='02725_memory_for_merges' and database=currentDatabase()) as uuid -SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where table_uuid=uuid and event_type='MergeParts'; +SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where database=currentDatabase() and table='02725_memory_for_merges' and event_type='MergeParts'; DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; From 08409059cc198873ffbf11060bfdabaa0c74f07f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 19 Jul 2023 18:46:20 +0800 Subject: [PATCH 498/871] 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 ae7952051e7..7533a99107b 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 @@ public: { if (!subquery_node->isDistinct()) 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 @@ public: { if (!subquery_node->hasGroupBy()) 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 @@ public: { 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()}}); } } @@ -135,7 +188,10 @@ public: void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - UniqToCountVisitor visitor(std::move(context)); + if (!context->getSettings().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 d7fa9f39441..e38e57f5cee 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 2ebbbf0000ce7f5767d754b0aee777a4255ab7b3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 19:13:45 +0800 Subject: [PATCH 499/871] Also need to fix aggregate projections --- .../QueryPlan/Optimizations/optimizeTree.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 01d192bb1f3..b13dda9a8f0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,6 +114,10 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + { /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); @@ -125,6 +129,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + /// Projection optimization relies on PK optimization if (optimization_settings.optimize_projection) num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); @@ -146,13 +151,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); - if (optimization_settings.optimize_projection) { - /// Normal projection optimization relies on PK optimization + /// Projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; From 94796f28adcd5b304b9fbc8a715462f4cfb1c1fd Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:24:16 +0800 Subject: [PATCH 500/871] ci fix --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings-formats.md | 2 +- src/Core/Settings.h | 2 +- .../queries/0_stateless/02811_csv_input_field_type_mismatch.sh | 0 4 files changed, 3 insertions(+), 3 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c20f304c346..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 5fac8df02d7..fb04ac23d3a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -971,7 +971,7 @@ a b ### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to set default value to column when CSV field deserialize failed on bad value +Allow to set default value to column when CSV field deserialization failed on bad value Default value: `false`. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 311813fb38f..309dfe0d2ec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,7 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh old mode 100644 new mode 100755 From 7837559dbfdc194f28681dda808bc06b6609dd8b Mon Sep 17 00:00:00 2001 From: Song Liyong Date: Wed, 12 Jul 2023 17:13:04 +0200 Subject: [PATCH 501/871] MaterializedMySQL: Support CREATE TABLE AS SELECT --- src/Core/MySQL/MySQLReplication.cpp | 11 +++++++ .../materialized_with_ddl.py | 29 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 3 files changed, 46 insertions(+) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..ab4a37d2466 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -121,6 +121,17 @@ namespace MySQLReplication { typ = QUERY_SAVEPOINT; } + + // https://dev.mysql.com/worklog/task/?id=13355 + // When doing query "CREATE TABLE xx AS SELECT", the binlog will be + // "CREATE TABLE ... START TRANSACTION", the DDL will be failed + // so, just ignore the "START TRANSACTION" suffix + if (query.ends_with("START TRANSACTION")) + { + auto pos = query.rfind("START TRANSACTION"); + if (pos > 0) + query.resize(pos); + } } void QueryEvent::dump(WriteBuffer & out) const 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 8cf9e67bf63..60326e422c9 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2336,3 +2336,32 @@ def named_collections(clickhouse_node, mysql_node, service_name): ) clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def create_table_as_select(clickhouse_node, mysql_node, service_name): + db = "create_table_as_select" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializeMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + mysql_node.query( + f"CREATE TABLE {db}.t1(a INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (1)") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\n", + ) + + mysql_node.query(f"CREATE TABLE {db}.t2(PRIMARY KEY(a)) AS SELECT * FROM {db}.t1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\nt2\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..f227c19e6b8 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -529,3 +529,9 @@ def test_named_collections(started_cluster, started_mysql_8_0, clickhouse_node): materialized_with_ddl.named_collections( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_create_table_as_select(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.create_table_as_select( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From dcf7ba25348f88bda0ef144ce068cc9005cb3ada Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:36:19 +0800 Subject: [PATCH 502/871] remove unuseful code --- docs/en/operations/settings/settings-formats.md | 3 +-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index fb04ac23d3a..b3bc3afafd3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -980,9 +980,8 @@ Default value: `false`. Query ```bash -echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 34d5b589591..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace DB From cc9da46efa2af4dfd4f8dfdfa84327f5f14a8630 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 20:11:03 +0800 Subject: [PATCH 503/871] ci fix --- .../02811_csv_input_field_type_mismatch.reference | 10 +++++----- .../0_stateless/02811_csv_input_field_type_mismatch.sh | 2 +- .../0_stateless/data_csv/csv_with_bad_field_values.csv | 10 +++++----- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index 19c7956ba84..6abcc56bacc 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,5 +1,5 @@ -0 111 1970-01-01 2023-03-24 00:00:00 false -1 abc 2023-03-14 2023-03-14 11:22:33 true -2 c 1970-01-01 1970-01-01 08:00:00 false -4 888 2023-03-14 1970-06-03 14:43:53 false -5 bks 1970-01-01 2023-07-19 18:17:59 false +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index 3961664b9b3..30223329eca 100755 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv index faedd9b6705..e829cc0106a 100644 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -1,5 +1,5 @@ -1,abc,2023-03-14,2023-03-14 11:22:33,true -2,c,ab,2023,false -bc,111,ab,2023-03-24,ban -4,888,2023-03-14,13243433,false -5,bks,2023-03,1689761879,abdd \ No newline at end of file +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From 0c86df519ffa8921b6c546b304705366838dfa21 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jul 2023 12:41:25 +0000 Subject: [PATCH 504/871] Fix unspported disks in Keeper --- src/Coordination/KeeperContext.cpp | 31 ++++++++++++++++++- src/Disks/DiskSelector.cpp | 5 ++- src/Disks/DiskSelector.h | 3 +- .../configs/enable_keeper.xml | 4 +++ tests/integration/test_keeper_disks/test.py | 12 ++++++- 5 files changed, 51 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 408344ee67f..32f8b98a7ed 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -41,9 +41,38 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) initializeDisks(config); } +namespace +{ + +bool diskValidator(const Poco::Util::AbstractConfiguration & config, const std::string & disk_config_prefix) +{ + const auto disk_type = config.getString(disk_config_prefix + ".type", "local"); + + using namespace std::literals; + static constexpr std::array supported_disk_types + { + "s3"sv, + "s3_plain"sv, + "local"sv + }; + + if (std::all_of( + supported_disk_types.begin(), + supported_disk_types.end(), + [&](const auto supported_type) { return disk_type != supported_type; })) + { + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Disk type '{}' is not supported for Keeper", disk_type); + return false; + } + + return true; +} + +} + void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & config) { - disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); + disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator); log_storage = getLogsPathFromConfig(config); diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index e51f79867b5..415e10a55fc 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -27,7 +27,7 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); @@ -46,6 +46,9 @@ void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, auto disk_config_prefix = config_prefix + "." + disk_name; + if (disk_validator && !disk_validator(config, disk_config_prefix)) + continue; + disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks)); } if (!has_default_disk) diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 58adeb953db..c91c3acb3bd 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -23,7 +23,8 @@ public: DiskSelector() = default; DiskSelector(const DiskSelector & from) = default; - void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + using DiskValidator = std::function; + void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator = {}); DiskSelectorPtr updateFromConfig( const Poco::Util::AbstractConfiguration & config, diff --git a/tests/integration/test_keeper_disks/configs/enable_keeper.xml b/tests/integration/test_keeper_disks/configs/enable_keeper.xml index 5814979229c..50d0329637a 100644 --- a/tests/integration/test_keeper_disks/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_disks/configs/enable_keeper.xml @@ -1,6 +1,10 @@ + + hdfs + hdfs://hdfs1:9000/ + local /var/lib/clickhouse/coordination/logs/ diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py index 11bb215be54..86682bcde01 100644 --- a/tests/integration/test_keeper_disks/test.py +++ b/tests/integration/test_keeper_disks/test.py @@ -9,7 +9,11 @@ import os CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, + with_minio=True, + with_hdfs=True, ) from kazoo.client import KazooClient, KazooState @@ -117,6 +121,12 @@ def get_local_snapshots(): return get_local_files("/var/lib/clickhouse/coordination/snapshots") +def test_supported_disk_types(started_cluster): + node.stop_clickhouse() + node.start_clickhouse() + node.contains_in_log("Disk type 'hdfs' is not supported for Keeper") + + def test_logs_with_disks(started_cluster): setup_local_storage(started_cluster) From 7b3564f96aa44bde8aa33914930ca3bbf1c5f52e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:44:59 +0200 Subject: [PATCH 505/871] Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" --- docs/en/interfaces/formats.md | 1 - .../operations/settings/settings-formats.md | 22 ---------- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - .../Formats/Impl/CSVRowInputFormat.cpp | 42 +------------------ .../Formats/Impl/CSVRowInputFormat.h | 2 - ...11_csv_input_field_type_mismatch.reference | 5 --- .../02811_csv_input_field_type_mismatch.sh | 13 ------ .../data_csv/csv_with_bad_field_values.csv | 5 --- 10 files changed, 2 insertions(+), 91 deletions(-) delete mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference delete mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ddf4ab3f78e..ed2f010a632 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,6 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index c8adc83d3ad..0915c51806a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,28 +989,6 @@ Result a b ``` -### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} - -Allow to set default value to column when CSV field deserialization failed on bad value - -Default value: `false`. - -**Examples** - -Query - -```bash -./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -./clickhouse local -q "select * from test_tbl" -``` - -Result - -```text -a 0 1971-01-01 -``` - ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a4cb0c2dbd9..730b6ab80ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,7 +874,6 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6e3e086859b..8eacc7acc97 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,7 +73,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e321e5264ca..af90e4462dd 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,7 +152,6 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 244b906549e..79ce2549b4d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -316,54 +315,17 @@ bool CSVFormatReader::readField( return false; } - if (format_settings.csv.use_default_on_bad_values) - return readFieldOrDefault(column, type, serialization); - return readFieldImpl(*buf, column, type, serialization); -} - -bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, istr, format_settings); + serialization->deserializeTextCSV(column, *buf, format_settings); return true; } -bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ - String field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromString tmp_buf(field); - bool is_bad_value = false; - bool res = false; - - size_t col_size = column.size(); - try - { - res = readFieldImpl(tmp_buf, column, type, serialization); - /// Check if we parsed the whole field successfully. - if (!field.empty() && !tmp_buf.eof()) - is_bad_value = true; - } - catch (const Exception &) - { - is_bad_value = true; - } - - if (!is_bad_value) - return res; - - if (column.size() == col_size + 1) - column.popBack(1); - column.insertDefault(); - return false; -} - void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 7b1a1fc433d..8ccf04feed3 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,8 +89,6 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } - bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); - bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference deleted file mode 100644 index 6abcc56bacc..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ /dev/null @@ -1,5 +0,0 @@ -0 111 1970-01-01 false -1 abc 2023-03-14 true -2 c 1970-01-01 false -4 888 2023-03-14 false -5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh deleted file mode 100755 index 30223329eca..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -# NOTE: this sh wrapper is required because of shell_config - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" -cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -$CLICKHOUSE_CLIENT -q "select * from test_tbl" -$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv deleted file mode 100644 index e829cc0106a..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ /dev/null @@ -1,5 +0,0 @@ -1,abc,2023-03-14,true -2,c,ab,false -bc,111,ab,ban -4,888,2023-03-14,false -5,bks,2023-03,abdd \ No newline at end of file From f0026af1893772a7a14f21d11cc88307ba07500a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:51:11 +0200 Subject: [PATCH 506/871] Revert "Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed"" --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 22 ++++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/CSVRowInputFormat.cpp | 42 ++++++++++++++++++- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 5 +++ .../02811_csv_input_field_type_mismatch.sh | 13 ++++++ .../data_csv/csv_with_bad_field_values.csv | 5 +++ 10 files changed, 91 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference create mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,6 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..c8adc83d3ad 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,6 +989,28 @@ Result a b ``` +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} + +Allow to set default value to column when CSV field deserialization failed on bad value + +Default value: `false`. + +**Examples** + +Query + +```bash +./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +./clickhouse local -q "select * from test_tbl" +``` + +Result + +```text +a 0 1971-01-01 +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 730b6ab80ed..a4cb0c2dbd9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,6 +874,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8eacc7acc97..6e3e086859b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,6 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index af90e4462dd..e321e5264ca 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,6 +152,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 79ce2549b4d..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -315,17 +316,54 @@ bool CSVFormatReader::readField( return false; } + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); + serialization->deserializeTextCSV(column, istr, format_settings); return true; } +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + + size_t col_size = column.size(); + try + { + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; + } + catch (const Exception &) + { + is_bad_value = true; + } + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; +} + void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference new file mode 100644 index 00000000000..6abcc56bacc --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -0,0 +1,5 @@ +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh new file mode 100755 index 00000000000..30223329eca --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..e829cc0106a --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From bdaf82922a599ab5abb123b4bbd1f41249543ec7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 12:54:25 +0000 Subject: [PATCH 507/871] Use formatAST() --- src/Interpreters/Cache/QueryCache.cpp | 9 ++++----- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 1d1543844a2..a6c509e8bb1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -115,12 +116,10 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } -String queryStringFromAst(ASTPtr ast) +String queryStringFromAST(ASTPtr ast) { WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); + formatAST(*ast, buf, /*hilite*/ false, /*one_line*/ true, /*show_secrets*/ false); return buf.str(); } @@ -138,7 +137,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) - , query_string(queryStringFromAst(ast_)) + , query_string(queryStringFromAST(ast_)) { } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index fca8ea0aa35..aa1afe17c75 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -4,9 +4,9 @@ namespace DB { -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); + IAST::FormatSettings settings(buf, one_line, show_secrets); settings.hilite = hilite; ast.format(settings); diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 28af2400a4c..ebd284fc18a 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -11,7 +11,7 @@ class WriteBuffer; /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); String serializeAST(const IAST & ast, bool one_line = true); From 9c4005b33fc74ab4ec3b68ebc877fdda499e8932 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 19 Jul 2023 15:12:47 +0200 Subject: [PATCH 508/871] Add logging about all found workflows for merge_pr.py --- tests/ci/merge_pr.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 14844ed9b25..35b0614b01f 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -246,6 +246,12 @@ def main(): if args.check_running_workflows: workflows = get_workflows_for_head(repo, pr.head.sha) + logging.info( + "The PR #%s has following workflows:\n%s", + pr.number, + "\n".join(f"{wf.html_url}: status is {wf.status}" for wf in workflows), + ) + workflows_in_progress = [wf for wf in workflows if wf.status != "completed"] # At most one workflow in progress is fine. We check that there no # cases like, e.g. PullRequestCI and DocksCheck in progress at once From 53818dde8cef7dd573217fa049d01b233a076ac2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 15:22:25 +0200 Subject: [PATCH 509/871] MergeTree/ReplicatedMergeTree should use server timezone for log entries Otherwise session_timezone/use_client_time_zone will break things Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index cac26c5ac23..4dbccb91620 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -61,7 +61,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP { auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); *out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time) << "\n"; + << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; commands.writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index ac956433eab..9eb8b6ce24c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -48,7 +48,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const format_version = std::max(format_version, FORMAT_WITH_LOG_ENTRY_ID); out << "format version: " << format_version << "\n" - << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << '\n' << "block_id: " << escape << block_id << '\n'; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1bbb246338c..e2c23ecfe85 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -12,7 +12,7 @@ namespace DB void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const { out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << "\n" << "block numbers count: " << block_numbers.size() << "\n"; From 7ad399cc6d97aaa356cbbe50f697b563d0a2f995 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:25:36 +0100 Subject: [PATCH 510/871] Rephrase ALL supported note I interpreted the note as 'ALL' is only supported before 23.4 I think this reordering makes it clearer --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index c3ddee07d0b..61c8a73673f 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -`ALL` is only applicable to the `RESTORE` command prior to version 23.4 of Clickhouse. +Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From ac18ca117807d3191c7dac505103e2d6191f3220 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 21:26:37 +0800 Subject: [PATCH 511/871] add test --- .../02813_system_events_and_metrics_add_alias.reference | 4 ++++ .../0_stateless/02813_system_events_and_metrics_add_alias.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference create mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference new file mode 100644 index 00000000000..93be2764a57 --- /dev/null +++ b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference @@ -0,0 +1,4 @@ +CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String,\n `name` String\n)\nENGINE = SystemMetrics\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +1 +CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String,\n `name` String\n)\nENGINE = SystemEvents\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +1 diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql new file mode 100644 index 00000000000..a2250608f8f --- /dev/null +++ b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql @@ -0,0 +1,4 @@ +show create table system.metrics; +select equals((select count() from system.metrics where name=metric) as r1, (select count() from system.metrics) as r2); +show create table system.events; +select equals((select count() from system.events where name=event) as r1, (select count() from system.events) as r2); \ No newline at end of file From c47f19303afc3e4f18ceddbfe7b8f7e7cc4622b3 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:27:21 +0100 Subject: [PATCH 512/871] Correct CH capitalisation --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 61c8a73673f..62f931a76b4 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. +Prior to version 23.4 of ClickHouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From 688b55b6ff80ee333ab9ef318d42937d5b5d3064 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:29:07 +0000 Subject: [PATCH 513/871] Try to fix test, rename arg --- src/Common/SystemLogBase.cpp | 9 +++++---- src/Common/SystemLogBase.h | 4 ++-- src/Loggers/Loggers.cpp | 6 +++--- tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index bed6d661db7..8cf8103e1c7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -41,9 +41,9 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_) - : log(&Poco::Logger::get(name_)) + : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) {} @@ -120,6 +120,7 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { + uint64_t this_thread_requested_offset; { @@ -204,10 +205,10 @@ void SystemLogQueue::shutdown() template SystemLogBase::SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) + : queue(queue_ ? queue_ : std::make_shared>(table_name_, flush_interval_milliseconds_)) { } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0ac376769ad..3716584be24 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -80,7 +80,7 @@ class SystemLogQueue public: SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_); void shutdown(); @@ -130,7 +130,7 @@ public: using Self = SystemLogBase; SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 4cc74902ee1..85a8152602f 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -255,10 +255,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log #ifndef WITHOUT_TEXT_LOG if (config.has("text_log")) { - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + String text_log_level_str = config.getString("text_log.level", "trace"); + int text_log_level = Poco::Logger::parseLevel(text_log_level_str); size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds", - DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); + DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level); } #endif diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql old mode 100755 new mode 100644 index 8ef78945a72..e007f58189e --- a/tests/queries/0_stateless/02813_starting_in_text_log.sql +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -1,2 +1,2 @@ SYSTEM FLUSH LOGS; -SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Starting ClickHouse%'; From 3225c30f730939ee688013d776b7f8651982c4a2 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 21:29:07 +0800 Subject: [PATCH 514/871] fix test --- .../0_stateless/02117_show_create_table_system.reference | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 3c221d6a473..4285908fa53 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -262,7 +262,8 @@ CREATE TABLE system.events ( `event` String, `value` UInt64, - `description` String + `description` String, + `name` String ) ENGINE = SystemEvents COMMENT 'SYSTEM TABLE is built on the fly.' @@ -382,7 +383,8 @@ CREATE TABLE system.metrics ( `metric` String, `value` Int64, - `description` String + `description` String, + `name` String ) ENGINE = SystemMetrics COMMENT 'SYSTEM TABLE is built on the fly.' From 9f7e40e8e57cc5e8c997dff16b5c6645283ffcb3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:43:22 +0000 Subject: [PATCH 515/871] Remove empty line --- src/Common/SystemLogBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 8cf8103e1c7..294ba09e375 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -120,7 +120,6 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { - uint64_t this_thread_requested_offset; { From 777026e42e00311c4a751e8899fac407d6c8b874 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 19 Jul 2023 13:43:40 +0000 Subject: [PATCH 516/871] Fix test --- src/Functions/s2RectAdd.cpp | 8 ++++---- src/Functions/s2RectContains.cpp | 8 ++++---- src/Functions/s2RectIntersection.cpp | 4 ++-- src/Functions/s2RectUnion.cpp | 6 +++--- tests/queries/0_stateless/01849_geoToS2.reference | 1 - tests/queries/0_stateless/01849_geoToS2.sql | 2 +- .../0_stateless/02224_s2_test_const_columns.reference | 2 +- tests/queries/0_stateless/02224_s2_test_const_columns.sql | 2 +- 8 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index 9266f4ae1a7..0be304234cb 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -118,14 +118,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); rect.AddPoint(point.ToPoint()); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index aed123ce8ee..898e12a6466 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -111,14 +111,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); dst_data.emplace_back(rect.Contains(point.ToLatLng())); } diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index ffe26d171d0..f0cc02de9d9 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -133,8 +133,8 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " "Also, if either the latitude or longitude bound is empty then both must be."); S2LatLngRect rect_intersection = rect1.Intersection(rect2); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index 472b30c2d55..a5cedd35812 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -131,9 +131,9 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); S2LatLngRect rect_union = rect1.Union(rect2); diff --git a/tests/queries/0_stateless/01849_geoToS2.reference b/tests/queries/0_stateless/01849_geoToS2.reference index 08d76978791..a8196994361 100644 --- a/tests/queries/0_stateless/01849_geoToS2.reference +++ b/tests/queries/0_stateless/01849_geoToS2.reference @@ -39,4 +39,3 @@ Checking s2 index generation. (74.0061,-68.32124) (74.0061,-68.32124) ok (10.61077,-64.1841) (10.61077,-64.1841) ok (-89.81096,-57.01398) (-89.81096,-57.01398) ok -4864204703484167331 diff --git a/tests/queries/0_stateless/01849_geoToS2.sql b/tests/queries/0_stateless/01849_geoToS2.sql index abd084a2b19..e997fec14e5 100644 --- a/tests/queries/0_stateless/01849_geoToS2.sql +++ b/tests/queries/0_stateless/01849_geoToS2.sql @@ -44,7 +44,7 @@ SELECT first, second, result FROM ( SELECT s2ToGeo(toUInt64(-1)); -- { serverError 36 } SELECT s2ToGeo(nan); -- { serverError 43 } -SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); +SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } SELECT geoToS2(nan, nan); -- { serverError 43 } SELECT geoToS2(-inf, 1.1754943508222875e-38); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.reference b/tests/queries/0_stateless/02224_s2_test_const_columns.reference index 9982596f097..20f32ec6be7 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.reference +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.reference @@ -16,4 +16,4 @@ (5179062030687166815,5177056748191934217) (5179062030687166815,5177057445452335297) (5178914411069187297,5177056748191934217) -(6304347505408739331,8070450532247928833) +(5178914411069187297,5177912432982045463) diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.sql b/tests/queries/0_stateless/02224_s2_test_const_columns.sql index f33a7f2b696..1d3e51065b5 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.sql +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.sql @@ -9,4 +9,4 @@ SELECT s2CapUnion(3814912406305146967, toFloat64(1), 1157347770437378819, toFloa SELECT s2RectAdd(5178914411069187297, 5177056748191934217, arrayJoin([5179056748191934217,5177914411069187297])); SELECT s2RectContains(5179062030687166815, 5177056748191934217, arrayJoin([5177914411069187297, 5177914411069187297])); SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217, 5177914411069187297])); -SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,1157347770437378819])); +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,5177914411069187297])); From 53500be941bc1d63ef85c3b5afb6bcc01103fb85 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 19 Jul 2023 17:03:00 +0200 Subject: [PATCH 517/871] Updated fix of multiple usage in parameterized view to support cte by not adding column which is previously added --- src/Interpreters/ActionsVisitor.cpp | 22 +++++++------------ ...zed_view_with_cte_multiple_usage.reference | 2 ++ ...meterized_view_with_cte_multiple_usage.sql | 16 ++++++++++++++ 3 files changed, 26 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference create mode 100755 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index efab11003f5..8b10df516dc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1202,22 +1202,16 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & else if (data.is_create_parameterized_view && query_parameter) { const auto data_type = DataTypeFactory::instance().get(query_parameter->type); - /// Use getUniqueName() to allow multiple use of query parameter in the query: - /// - /// CREATE VIEW view AS - /// SELECT * - /// FROM system.one - /// WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 - /// ^^ ^^ - /// - /// NOTE: query in the VIEW will not be modified this is needed - /// only during analysis for CREATE VIEW to avoid duplicated - /// column names. - ColumnWithTypeAndName column(data_type, data.getUniqueName("__" + query_parameter->getColumnName())); - data.addColumn(column); + /// During analysis for CREATE VIEW of a parameterized view, if parameter is + /// used multiple times, column is only added once + if (!data.hasColumn(query_parameter->name)) + { + ColumnWithTypeAndName column(data_type, query_parameter->name); + data.addColumn(column); + } argument_types.push_back(data_type); - argument_names.push_back(column.name); + argument_names.push_back(query_parameter->name); } else { diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference new file mode 100644 index 00000000000..004d27bacad --- /dev/null +++ b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference @@ -0,0 +1,2 @@ +3 2 +3 2 3 diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql new file mode 100755 index 00000000000..d56d9c4e181 --- /dev/null +++ b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql @@ -0,0 +1,16 @@ +create view test_param_view as +with {param_test_val:UInt8} as param_test_val +select param_test_val, + arrayCount((a)->(a < param_test_val), t.arr) as cnt1 +from (select [1,2,3,4,5] as arr) t; + +select * from test_param_view(param_test_val = 3); + +create view test_param_view2 as +with {param_test_val:UInt8} as param_test_val +select param_test_val, + arrayCount((a)->(a < param_test_val), t.arr) as cnt1, + arrayCount((a)->(a < param_test_val+1), t.arr) as cnt2 +from (select [1,2,3,4,5] as arr) t; + +select * from test_param_view2(param_test_val = 3); \ No newline at end of file From b34655e74310dba07c25b06cac817168b6012907 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 19 Jul 2023 17:05:03 +0200 Subject: [PATCH 518/871] 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 53fac578fca..04799a08e37 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 519/871] 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 04799a08e37..06f5330f6d9 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 13d1e21da820dd97ddb624eb7671ca2fee86d530 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 19 Jul 2023 18:26:26 +0200 Subject: [PATCH 520/871] Fixed test file permissions --- .../02818_parameterized_view_with_cte_multiple_usage.sql | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql old mode 100755 new mode 100644 From 90be5e6160f2342121dfe2d014f4d247a4efa39c Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:04:43 +0100 Subject: [PATCH 521/871] Remove reference to `TIMEOUT` in live views documentation Temporary live views were removed in #42173, but the documentation was not fully updated to reflect the change. --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 10b15638152..11026340a0f 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -97,7 +97,7 @@ This is an experimental feature that may change in backwards-incompatible ways i ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` Live views store result of the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query and are updated any time the result of the query changes. Query result as well as partial result needed to combine with new data are stored in memory providing increased performance for repeated queries. Live views can provide push notifications when query result changes using the [WATCH](../../../sql-reference/statements/watch.md) query. From 02fe735b768e2d171191091c402f4732ace4669e Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:12:35 +0100 Subject: [PATCH 522/871] fix in other other languages --- docs/ru/sql-reference/statements/create/view.md | 2 +- docs/zh/sql-reference/statements/create/view.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index d3846aac289..1a60dc0716c 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -73,7 +73,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na Чтобы использовать `LIVE VIEW` и запросы `WATCH`, включите настройку [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view). ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` `LIVE VIEW` хранит результат запроса [SELECT](../../../sql-reference/statements/select/index.md), указанного при создании, и обновляется сразу же при изменении этого результата. Конечный результат запроса и промежуточные данные, из которых формируется результат, хранятся в оперативной памяти, и это обеспечивает высокую скорость обработки для повторяющихся запросов. LIVE-представления могут отправлять push-уведомления при изменении результата исходного запроса `SELECT`. Для этого используйте запрос [WATCH](../../../sql-reference/statements/watch.md). diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index 8ce2d20a10c..bce0994ecd2 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -72,7 +72,7 @@ ClickHouse 中的物化视图更像是插入触发器。 如果视图查询中 使用[allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view)设置启用实时视图和`WATCH`查询的使用。 输入命令`set allow_experimental_live_view = 1`。 ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` 实时视图存储相应[SELECT](../../../sql-reference/statements/select/index.md)查询的结果,并在查询结果更改时随时更新。 查询结果以及与新数据结合所需的部分结果存储在内存中,为重复查询提供更高的性能。当使用[WATCH](../../../sql-reference/statements/watch.md)查询更改查询结果时,实时视图可以提供推送通知。 From dbdac5d823d431fb34405649f7125e76c88f1f05 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 19 Jul 2023 19:34:49 +0000 Subject: [PATCH 523/871] 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 2e049dbc936..763a7cc4286 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 41744cc59f9..f07cdca4b5a 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 c9301c1f0a3..50ca5981cf1 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 524/871] 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 1880af4ccd2..5e6e7efd938 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: 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 @@ protected: 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 @@ private: 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 5543e02b279..91a82868dcf 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 8b969f87a4d..db5d1a2f5d9 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 8f304f0aab6..ffa72d841be 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 6d1c0f7aafa..eea801ce65e 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 e6953afe68e..4cfefbc5527 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 525/871] 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 91a82868dcf..9bad1b39a87 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 26d4975954f..9f2bf177c96 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 2ab43f1b895..96f36f03949 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 e7b8767585e748d91796e669b871d40546c40bc8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:14:05 +0200 Subject: [PATCH 526/871] Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness --- tests/queries/0_stateless/02125_many_mutations_2.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index 5b779c1b276..819ac8c9524 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From ff235e0f3078f6c27a9a1ab1383a91378313ab77 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 05:41:39 +0000 Subject: [PATCH 527/871] Turn off log in queue, fix data race --- src/Common/SystemLogBase.cpp | 9 +++++++-- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/TextLog.h | 6 ++++-- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 294ba09e375..d1845a292b9 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -42,10 +42,14 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_) + size_t flush_interval_milliseconds_, + bool turn_off_logger_) : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) -{} +{ + if (turn_off_logger_) + log->setLevel(0); +} static thread_local bool recursive_push_call = false; @@ -197,6 +201,7 @@ SystemLogQueue::Index SystemLogQueue::pop(std::vector void SystemLogQueue::shutdown() { + std::unique_lock lock(mutex); is_shutdown = true; /// Tell thread to shutdown. flush_event.notify_all(); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 3716584be24..f6e4a579edf 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -81,7 +81,8 @@ class SystemLogQueue public: SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + bool turn_off_logger_ = false); void shutdown(); diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 0febce03abc..60ca11632aa 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -40,6 +40,8 @@ struct TextLogElement class TextLog : public SystemLog { public: + using Queue = SystemLogQueue; + TextLog( ContextPtr context_, const String & database_name_, @@ -47,9 +49,9 @@ public: const String & storage_def_, size_t flush_interval_milliseconds_); - static std::shared_ptr> getLogQueue(size_t flush_interval_milliseconds) + static std::shared_ptr getLogQueue(size_t flush_interval_milliseconds) { - static std::shared_ptr> queue = std::make_shared>("text_log", flush_interval_milliseconds); + static std::shared_ptr queue = std::make_shared("text_log", flush_interval_milliseconds, true); return queue; } }; From 27ca367b2cb349391946c45d7e3d22fe6d543c42 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jul 2023 10:01:07 +0200 Subject: [PATCH 528/871] ZooKeeperRetriesControl rethrows with original callstack. --- src/Storages/MergeTree/ZooKeeperRetries.h | 44 ++++++++++++++++++----- 1 file changed, 35 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index e55b04c27b3..512c0800de7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -72,7 +72,7 @@ public: if (!Coordination::isHardwareError(e.code)) throw; - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (...) { @@ -91,16 +91,16 @@ public: } catch (const zkutil::KeeperException & e) { - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (const Exception & e) { - setUserError(e.code(), e.what()); + setUserError(std::current_exception(), e.code(), e.what()); } return false; } - void setUserError(int code, std::string message) + void setUserError(std::exception_ptr exception, int code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -113,16 +113,28 @@ public: iteration_succeeded = false; user_error.code = code; user_error.message = std::move(message); + user_error.exception = exception; keeper_error = KeeperError{}; } + template + void setUserError(std::exception_ptr exception, int code, fmt::format_string fmt, Args &&... args) + { + setUserError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setUserError(int code, std::string message) + { + setUserError(std::make_exception_ptr(Exception::createDeprecated(message, code)), code, message); + } + template void setUserError(int code, fmt::format_string fmt, Args &&... args) { setUserError(code, fmt::format(fmt, std::forward(args)...)); } - void setKeeperError(Coordination::Error code, std::string message) + void setKeeperError(std::exception_ptr exception, Coordination::Error code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -135,9 +147,21 @@ public: iteration_succeeded = false; keeper_error.code = code; keeper_error.message = std::move(message); + keeper_error.exception = exception; user_error = UserError{}; } + template + void setKeeperError(std::exception_ptr exception, Coordination::Error code, fmt::format_string fmt, Args &&... args) + { + setKeeperError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + setKeeperError(std::make_exception_ptr(zkutil::KeeperException(message, code)), code, message); + } + template void setKeeperError(Coordination::Error code, fmt::format_string fmt, Args &&... args) { @@ -163,12 +187,14 @@ private: using Code = Coordination::Error; Code code = Code::ZOK; std::string message; + std::exception_ptr exception; }; struct UserError { int code = ErrorCodes::OK; std::string message; + std::exception_ptr exception; }; bool canTry() @@ -232,11 +258,11 @@ private: void throwIfError() const { - if (user_error.code != ErrorCodes::OK) - throw Exception::createDeprecated(user_error.message, user_error.code); + if (user_error.exception) + std::rethrow_exception(user_error.exception); - if (keeper_error.code != KeeperError::Code::ZOK) - throw zkutil::KeeperException(keeper_error.message, keeper_error.code); + if (keeper_error.exception) + std::rethrow_exception(keeper_error.exception); } void logLastError(std::string_view header) From 6a21995b2097e747a28a23333e651208c25f0224 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 20 Jul 2023 10:42:19 +0200 Subject: [PATCH 529/871] Added test to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..9a9412e55db 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,4 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02818_parameterized_view_with_cte_multiple_usage From c7ab6e908adf2a088ad41e00ea2bfad5ea16526a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 08:55:22 +0000 Subject: [PATCH 530/871] Move tode to to try to make the diff simpler --- src/Common/SystemLogBase.cpp | 60 ++++++++++++++++++------------------ 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index d1845a292b9..ed5ffd78a7b 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -121,6 +121,36 @@ void SystemLogQueue::push(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) +{ + // Use an arbitrary timeout to avoid endless waiting. 60s proved to be + // too fast for our parallel functional tests, probably because they + // heavily load the disk. + const int timeout_seconds = 180; + std::unique_lock lock(mutex); + bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + { + return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; + }); + + if (!result) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", + toString(timeout_seconds), demangle(typeid(*this).name())); + } +} + template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { @@ -145,26 +175,6 @@ uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyw return this_thread_requested_offset; } -template -void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) -{ - // Use an arbitrary timeout to avoid endless waiting. 60s proved to be - // too fast for our parallel functional tests, probably because they - // heavily load the disk. - const int timeout_seconds = 180; - std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] - { - return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; - }); - - if (!result) - { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", - toString(timeout_seconds), demangle(typeid(*this).name())); - } -} - template void SystemLogQueue::confirm(uint64_t to_flush_end) { @@ -229,16 +239,6 @@ void SystemLogBase::add(const LogElement & element) queue->push(element); } -template -void SystemLogBase::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - template void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } From 16cc00784217574bfa4b434936b25f24c531e542 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 20 Jul 2023 09:21:18 +0000 Subject: [PATCH 531/871] Fix table ad variable name --- tests/integration/test_storage_kafka/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 27a315b53bb..7013f0198f3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -865,7 +865,7 @@ def test_kafka_formats(kafka_cluster): expected_rows_count = raw_expected.count("\n") instance.query_with_retry( - f"SELECT * FROM test.kafka_data_{list(all_formats.keys())[-1]}_mv;", + f"SELECT * FROM test.kafka_{list(all_formats.keys())[-1]}_mv;", retry_count=30, sleep_time=1, check_callback=lambda res: res.count("\n") == expected_rows_count, @@ -3798,7 +3798,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): format_name=format_name ) ) - expected = pre_formatted_expected.format( + expected = raw_expected.format( topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], From 067e3caa2c43ed981a7c598c45668f37b0ac32c6 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 18:13:19 +0800 Subject: [PATCH 532/871] Remove constants from description_sorted_. --- src/Processors/Transforms/FinishSortingTransform.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 066928446f2..744d035d0ee 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,15 +38,11 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); - size_t num_columns = header.columns(); - ColumnNumbers map(num_columns, num_columns); for (const auto & column_description : description_sorted_) { - auto old_pos = header.getPositionByName(column_description.column_name); - auto new_pos = map[old_pos]; + auto pos = header.getPositionByName(column_description.column_name); - if (new_pos < num_columns) - { + if (!const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From 0ba97eeea597ad027c375cf292419dd555a9cb73 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 15 Jun 2023 08:05:47 +0800 Subject: [PATCH 533/871] wip: grace hash join support full & right join --- docs/en/operations/settings/settings.md | 2 + src/Interpreters/GraceHashJoin.cpp | 31 +++++++-- src/Interpreters/GraceHashJoin.h | 3 +- .../Transforms/JoiningTransform.cpp | 65 +++++++++++++++++-- src/Processors/Transforms/JoiningTransform.h | 24 ++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...01721_join_implicit_cast_long.reference.j2 | 40 ------------ .../01721_join_implicit_cast_long.sql.j2 | 1 - .../02273_full_sort_join.reference.j2 | 18 +---- .../0_stateless/02273_full_sort_join.sql.j2 | 4 +- ...274_full_sort_join_nodistinct.reference.j2 | 34 +--------- .../02274_full_sort_join_nodistinct.sql.j2 | 6 +- .../02275_full_sort_join_long.reference | 24 ++++++- .../02275_full_sort_join_long.sql.j2 | 9 +-- 14 files changed, 138 insertions(+), 125 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 22aeecf4335..580b51a984d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -537,6 +537,8 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. + - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index edf604bc0b4..f94453293f6 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,8 +301,10 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { + bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); + auto kind = table_join->kind(); + return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -322,7 +324,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; - bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -494,17 +495,30 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const +/// Each bucket are handled by the following steps +/// 1. build hash_join by the right side blocks. +/// 2. join left side with the hash_join, +/// 3. read right non-joined blocks from hash_join. +/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. +/// +/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +/// only one processor could take the non-joined blocks from right stream, and ensure all rows from +/// left stream have been emitted before this. +IBlocksStreamPtr +GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - /// We do no support returning non joined blocks here. - /// TODO: They _should_ be reported by getDelayedBlocks instead - return nullptr; + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) + explicit DelayedBlocks( + size_t current_bucket_, + Buckets buckets_, + InMemoryJoinPtr hash_join_, + const Names & left_key_names_, + const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) @@ -522,12 +536,15 @@ public: do { + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. + // There is a lock inside left_reader.read() . block = left_reader.read(); if (!block) { return {}; } + // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bce04ee6b04..ce519892b0e 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,7 +13,6 @@ namespace DB { - class TableJoin; class HashJoin; @@ -79,7 +78,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 49b90d04b81..f1ceefbf229 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -311,8 +310,16 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) + , left_header(left_header_) + , output_header(output_header_) + , max_block_size(max_block_size_) + , join(join_) { } @@ -365,6 +372,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else @@ -387,11 +395,24 @@ void DelayedJoinedBlocksWorkerTransform::work() if (!task) return; - Block block = task->delayed_blocks->next(); + Block block; + if (!left_delayed_stream_finished) + { + block = task->delayed_blocks->next(); + if (!block) + { + left_delayed_stream_finished = true; + block = nextNonJoinedBlock(); + } + } + else + { + block = nextNonJoinedBlock(); + } if (!block) { - task.reset(); + resetTask(); return; } @@ -400,6 +421,31 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } +void DelayedJoinedBlocksWorkerTransform::resetTask() +{ + task.reset(); + left_delayed_stream_finished = false; + non_joined_delayed_stream = nullptr; +} + +Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() +{ + // Before read from non-joined stream, all blocks in left file reader must have been joined. + // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before + // the all blocks in left file reader have been finished, since the used flags are incomplete. + // To make only one processor could read from non-joined stream seems be a easy way. + if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) + { + non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + } + + if (non_joined_delayed_stream) + { + return non_joined_delayed_stream->next(); + } + return {}; +} + DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -433,6 +479,9 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (finished) { + // Since have memory limit, cannot handle all buckets parallelly by different + // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. + // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) { if (output.isFinished()) @@ -448,10 +497,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { + // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform + // could read right non-joined blocks from the join. + auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); + auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..10b413ed4e5 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,9 +116,14 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) + : delayed_blocks(std::move(delayed_blocks_)) + , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) + { + } IBlocksStreamPtr delayed_blocks = nullptr; + JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -147,7 +152,11 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform(Block output_header); + explicit DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -155,10 +164,19 @@ public: void work() override; private: + Block left_header; + Block output_header; + size_t max_block_size; + JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - bool finished = false; + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + bool left_delayed_stream_finished = false; + IBlocksStreamPtr non_joined_delayed_stream = nullptr; + + void resetTask(); + Block nextNonJoinedBlock(); }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index dedf85e409c..ba98d725532 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(joined_header); + auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index e9f32087439..ae43aa7195c 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,7 +1,6 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -17,7 +16,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -30,7 +28,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -41,7 +38,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -49,7 +45,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -65,7 +60,6 @@ 8 8 0 9 9 0 10 10 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -78,7 +72,6 @@ 9 9 0 10 10 0 = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -89,7 +82,6 @@ 3 3 3 4 4 4 5 5 5 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -98,7 +90,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -114,7 +105,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -127,7 +117,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -138,7 +127,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -146,7 +134,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -162,7 +149,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -175,7 +161,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -186,7 +171,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -196,7 +180,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -205,13 +188,11 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -219,11 +200,9 @@ 1 1 1 -{% endif -%} {% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -239,7 +218,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -252,7 +230,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -263,7 +240,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -271,7 +247,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -287,7 +262,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -300,7 +274,6 @@ 9 9 \N 10 10 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -311,7 +284,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -320,7 +292,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -336,7 +307,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -349,7 +319,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -360,7 +329,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -368,7 +336,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -384,7 +351,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -397,7 +363,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -408,7 +373,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -418,7 +382,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -427,13 +390,11 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -442,5 +403,4 @@ 1 1 {% endif -%} -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index f5321939f28..38f71f4c5ec 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,7 +10,6 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 98bfd9d9b2b..0af4158e971 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -50,7 +50,6 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -64,7 +63,6 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -85,7 +83,6 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -99,7 +96,6 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 -{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -137,7 +133,6 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -150,7 +145,6 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -170,7 +164,6 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -183,7 +176,6 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 -{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -219,7 +211,6 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -233,7 +224,6 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -254,7 +244,6 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -268,7 +257,6 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -296,7 +284,6 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -309,7 +296,6 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -329,7 +315,6 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -342,5 +327,4 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 43f7354017c..6b6aa53836e 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -28,9 +28,7 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); - {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +38,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2cc6c6e85d6..df968e86e8d 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -108,7 +108,6 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -161,7 +160,6 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -177,7 +175,6 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -188,9 +185,7 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -243,9 +238,7 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -298,7 +291,6 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 -{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -315,7 +307,6 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -326,7 +317,6 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -341,7 +331,6 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -352,9 +341,7 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -372,9 +359,8 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% endif -%} -t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 2 5 0 @@ -409,7 +395,6 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -420,7 +405,6 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -435,7 +419,6 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -446,9 +429,7 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -466,9 +447,7 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 -{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -486,7 +465,6 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -503,7 +481,6 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -514,7 +491,6 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -529,7 +505,6 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -540,9 +515,7 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -560,9 +533,8 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% endif -%} -tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 3 4 4 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 613da65421e..f8eb4b1a53e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,7 +16,6 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -27,7 +26,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} @@ -47,9 +46,10 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} +{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} - +{% endif -%} {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 9ec06aea3e6..73482358d12 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,16 +41,34 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 7276e77dc16..621352f9c25 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,11 +22,6 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -{% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } -SELECT 'skipped'; -{% endif -%} -{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +35,6 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; -{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -58,9 +52,8 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; {{ is_implemented(join_algorithm) }} +; -{% endif -%} {% endfor -%} {% endfor -%} From 91dc6a35e17417a44de46d76c0f0214911615244 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 19 Jul 2023 09:18:16 +0800 Subject: [PATCH 534/871] update --- src/Interpreters/GraceHashJoin.cpp | 1 - .../Transforms/JoiningTransform.cpp | 19 +++++-------------- src/Processors/Transforms/JoiningTransform.h | 14 ++++---------- src/QueryPipeline/QueryPipelineBuilder.cpp | 5 ++++- 4 files changed, 13 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f94453293f6..5d72cf20740 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,7 +301,6 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { - bool is_asof = (table_join->strictness() == JoinStrictness::Asof); auto kind = table_join->kind(); return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f1ceefbf229..5480fea27a4 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -311,15 +311,10 @@ void FillingRightJoinSideTransform::work() DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_) + NonJoinedStreamBuilder non_joined_stream_builder_) : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) - , left_header(left_header_) - , output_header(output_header_) - , max_block_size(max_block_size_) - , join(join_) + , non_joined_stream_builder(std::move(non_joined_stream_builder_)) { } @@ -396,15 +391,12 @@ void DelayedJoinedBlocksWorkerTransform::work() return; Block block; - if (!left_delayed_stream_finished) + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + if (!task->delayed_blocks->isFinished()) { block = task->delayed_blocks->next(); - if (!block) - { - left_delayed_stream_finished = true; block = nextNonJoinedBlock(); - } } else { @@ -424,7 +416,6 @@ void DelayedJoinedBlocksWorkerTransform::work() void DelayedJoinedBlocksWorkerTransform::resetTask() { task.reset(); - left_delayed_stream_finished = false; non_joined_delayed_stream = nullptr; } @@ -436,7 +427,7 @@ Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() // To make only one processor could read from non-joined stream seems be a easy way. if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) { - non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + non_joined_delayed_stream = non_joined_stream_builder(); } if (non_joined_delayed_stream) diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 10b413ed4e5..5e7403dbbdb 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -152,11 +152,10 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: + using NonJoinedStreamBuilder = std::function; explicit DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_); + NonJoinedStreamBuilder non_joined_stream_builder_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -164,15 +163,10 @@ public: void work() override; private: - Block left_header; - Block output_header; - size_t max_block_size; - JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - - /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left - bool left_delayed_stream_finished = false; + /// For building a block stream to access the non-joined rows. + NonJoinedStreamBuilder non_joined_stream_builder; IBlocksStreamPtr non_joined_delayed_stream = nullptr; void resetTask(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index ba98d725532..553b18dd57b 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,10 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); + auto delayed = std::make_shared( + joined_header, + [left_header, joined_header, max_block_size, join]() + { return join->getNonJoinedBlocks(left_header, joined_header, max_block_size); }); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); From 7a0de384d498497fd026283a8232fcb8ed8ea5e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 16:46:51 +0000 Subject: [PATCH 535/871] Cosmetics --- src/Parsers/IAST.h | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..f6b7f91fec8 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -196,20 +196,23 @@ public: bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). - - // Newline or whitespace. - char nl_or_ws; + char nl_or_ws; /// Newline or whitespace. FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) - : ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_) + : ostr(ostr_) + , one_line(one_line_) + , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) - : ostr(ostr_), hilite(other.hilite), one_line(other.one_line), - always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style), - show_secrets(other.show_secrets) + : ostr(ostr_) + , hilite(other.hilite) + , one_line(other.one_line) + , always_quote_identifiers(other.always_quote_identifiers) + , identifier_quoting_style(other.identifier_quoting_style) + , show_secrets(other.show_secrets) { nl_or_ws = one_line ? ' ' : '\n'; } From 35a4fabc2d66ea28b3de3d77df4cfea4b91df870 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:03:04 +0000 Subject: [PATCH 536/871] Make IAST::FormatSettings more regular --- src/Parsers/IAST.cpp | 4 +++- src/Parsers/IAST.h | 3 +-- src/Parsers/formatAST.cpp | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..bf4d6fc9dec 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -170,7 +170,9 @@ size_t IAST::checkDepthImpl(size_t max_depth) const String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const { WriteBufferFromOwnString buf; - format({buf, one_line, show_secrets}); + FormatSettings settings(buf, one_line); + settings.show_secrets = show_secrets; + format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index f6b7f91fec8..13b2e5d9867 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -198,10 +198,9 @@ public: bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) + FormatSettings(WriteBuffer & ostr_, bool one_line_) : ostr(ostr_) , one_line(one_line_) - , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index aa1afe17c75..bc7faf4bd1d 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,9 +6,9 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line, show_secrets); + IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; - + settings.show_secrets = show_secrets; ast.format(settings); } From e5ec6a1523529db3d1b9d7f137997076c8c2adde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:21:03 +0000 Subject: [PATCH 537/871] Make IAST::FormatSettings more regular, pt. II --- src/Parsers/IAST.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 13b2e5d9867..8e2971d0355 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -191,8 +191,8 @@ public: struct FormatSettings { WriteBuffer & ostr; - bool hilite = false; bool one_line; + bool hilite = false; bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). @@ -207,13 +207,13 @@ public: FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) : ostr(ostr_) - , hilite(other.hilite) , one_line(other.one_line) + , hilite(other.hilite) , always_quote_identifiers(other.always_quote_identifiers) , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) + , nl_or_ws(other.nl_or_ws) { - nl_or_ws = one_line ? ' ' : '\n'; } void writeIdentifier(const String & name) const; From 25ddcc256b04de71e84935cb60a53190c114a494 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:31:53 +0000 Subject: [PATCH 538/871] Make IAST::FormatSettings more regular, pt. III --- src/Parsers/IAST.h | 22 ++++++++++++++----- src/Parsers/formatAST.cpp | 3 +-- src/Parsers/getInsertQuery.cpp | 4 +--- src/Parsers/tests/gtest_format_hiliting.cpp | 3 +-- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 +--- .../MeiliSearch/StorageMeiliSearch.cpp | 7 +++--- src/Storages/StorageDistributed.cpp | 6 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- .../transformQueryForExternalDatabase.cpp | 7 +++--- 9 files changed, 31 insertions(+), 28 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 8e2971d0355..d217876459f 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -192,17 +192,27 @@ public: { WriteBuffer & ostr; bool one_line; - bool hilite = false; - bool always_quote_identifiers = false; - IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; - bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). + bool hilite; + bool always_quote_identifiers; + IdentifierQuotingStyle identifier_quoting_style; + bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_) + explicit FormatSettings( + WriteBuffer & ostr_, + bool one_line_, + bool hilite_ = false, + bool always_quote_identifiers_ = false, + IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, + bool show_secrets_ = true) : ostr(ostr_) , one_line(one_line_) + , hilite(hilite_) + , always_quote_identifiers(always_quote_identifiers_) + , identifier_quoting_style(identifier_quoting_style_) + , show_secrets(show_secrets_) + , nl_or_ws(one_line ? ' ' : '\n') { - nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index bc7faf4bd1d..ae2c4a6fcad 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,8 +6,7 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); - settings.hilite = hilite; + IAST::FormatSettings settings(buf, one_line, hilite); settings.show_secrets = show_secrets; ast.format(settings); } diff --git a/src/Parsers/getInsertQuery.cpp b/src/Parsers/getInsertQuery.cpp index 6f52056dfe2..9d111b147bd 100644 --- a/src/Parsers/getInsertQuery.cpp +++ b/src/Parsers/getInsertQuery.cpp @@ -19,9 +19,7 @@ std::string getInsertQuery(const std::string & db_name, const std::string & tabl query.columns->children.emplace_back(std::make_shared(column.name)); WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; + IAST::FormatSettings settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true, /*identifier_quoting_style*/ quoting); query.IAST::format(settings); return buf.str(); } diff --git a/src/Parsers/tests/gtest_format_hiliting.cpp b/src/Parsers/tests/gtest_format_hiliting.cpp index d0ce8f2c897..a4c3ed86182 100644 --- a/src/Parsers/tests/gtest_format_hiliting.cpp +++ b/src/Parsers/tests/gtest_format_hiliting.cpp @@ -51,8 +51,7 @@ void compare(const String & expected, const String & query) ASTPtr ast = parseQuery(parser, query, 0, 0); WriteBufferFromOwnString write_buffer; - IAST::FormatSettings settings(write_buffer, true); - settings.hilite = true; + IAST::FormatSettings settings(write_buffer, true, true); ast->format(settings); ASSERT_PRED2(HiliteComparator::are_equal_with_hilites_removed, expected, write_buffer.str()); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..5cc13f45df4 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -86,9 +86,7 @@ static String formattedAST(const ASTPtr & ast) return {}; WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.hilite = false; - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); ast->format(ast_format_settings); return buf.str(); } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index 5d77fc080a4..aa8b437263a 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -62,9 +62,10 @@ ColumnsDescription StorageMeiliSearch::getTableStructureFromData(const MeiliSear String convertASTtoStr(ASTPtr ptr) { WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = IdentifierQuotingStyle::BackticksMySQL; - settings.always_quote_identifiers = IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ IdentifierQuotingStyle::BackticksMySQL); ptr->format(settings); return out.str(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..e02d7f32b98 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -906,8 +906,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers_=*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } @@ -968,8 +967,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5f20c497cb8..52f478d7729 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5074,8 +5074,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu String query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); query.IAST::format(ast_format_settings); query_str = buf.str(); } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 548b55749d7..375510e62bf 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -334,9 +334,10 @@ String transformQueryForExternalDatabaseImpl( dropAliases(select_ptr); WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = identifier_quoting_style; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ identifier_quoting_style); select->format(settings); From bd761c365a95e97f1a92638f145353d54a4f2db5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 18:02:09 +0000 Subject: [PATCH 539/871] Make serializeAST() more regular --- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Interpreters/Cache/QueryCache.h | 2 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 9 +++++---- src/Parsers/tests/gtest_Parser.cpp | 10 ++++++++-- src/Parsers/tests/gtest_dictionary_parser.cpp | 10 +++++----- .../Transforms/CheckConstraintsTransform.cpp | 4 ++-- src/Storages/ConstraintsDescription.cpp | 2 +- src/Storages/IndicesDescription.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 2 +- 11 files changed, 28 insertions(+), 21 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 81d5b7372f3..a9a0e972bd1 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -32,7 +32,7 @@ namespace /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); + auto disk_setting_string = serializeAST(function); disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index a67adcc86c9..c24b09c8e46 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -60,7 +60,7 @@ public: /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in - /// the SYSTEM.QUERY_CACHE. + /// SYSTEM.QUERY_CACHE. const String query_string; /// Ctor to construct a Key for writing into query cache. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 5acfe500b1d..398bea26b87 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -518,7 +518,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String static String getCleanQueryAst(const ASTPtr q, ContextPtr context) { - String res = serializeAST(*q, true); + String res = serializeAST(*q); if (auto * masker = SensitiveDataMasker::getInstance()) masker->wipeSensitiveData(res); diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index ae2c4a6fcad..9315279eae6 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -11,10 +11,10 @@ void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, ast.format(settings); } -String serializeAST(const IAST & ast, bool one_line) +String serializeAST(const IAST & ast) { WriteBufferFromOwnString buf; - formatAST(ast, buf, false, one_line); + formatAST(ast, buf, false, true); return buf.str(); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index ebd284fc18a..dd72a59b4a2 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -8,12 +8,13 @@ namespace DB class WriteBuffer; -/** Takes a syntax tree and turns it back into text. - * In case of INSERT query, the data will be missing. - */ +/// Takes a syntax tree and turns it into text. +/// Intended for pretty-printing (multi-line + hiliting). +/// In case of INSERT query, the data will be missing. void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); -String serializeAST(const IAST & ast, bool one_line = true); +/// Like formatAST() but intended for serialization w/o pretty-printing (single-line, no hiliting). +String serializeAST(const IAST & ast); inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast) { diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..a53de155355 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -64,7 +64,10 @@ TEST_P(ParserTest, parseQuery) if (std::string("CREATE USER or ALTER USER query") != parser->getName() && std::string("ATTACH access entity query") != parser->getName()) { - EXPECT_EQ(expected_ast, serializeAST(*ast->clone(), false)); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_EQ(expected_ast, formatted_ast); } else { @@ -75,7 +78,10 @@ TEST_P(ParserTest, parseQuery) } else { - EXPECT_TRUE(std::regex_match(serializeAST(*ast->clone(), false), std::regex(expected_ast))); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast))); } } } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 22484727ea2..c0a975f7a38 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -155,7 +155,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -201,7 +201,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -288,7 +288,7 @@ TEST(ParserDictionaryDDL, Formatting) ParserCreateDictionaryQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); - auto str = serializeAST(*create, true); + auto str = serializeAST(*create); EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)"); } @@ -303,7 +303,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop1->is_dictionary); EXPECT_EQ(drop1->getDatabase(), "test"); EXPECT_EQ(drop1->getTable(), "dict1"); - auto str1 = serializeAST(*drop1, true); + auto str1 = serializeAST(*drop1); EXPECT_EQ(input1, str1); String input2 = "DROP DICTIONARY IF EXISTS dict2"; @@ -314,7 +314,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop2->is_dictionary); EXPECT_EQ(drop2->getDatabase(), ""); EXPECT_EQ(drop2->getTable(), "dict2"); - auto str2 = serializeAST(*drop2, true); + auto str2 = serializeAST(*drop2); EXPECT_EQ(input2, str2); } diff --git a/src/Processors/Transforms/CheckConstraintsTransform.cpp b/src/Processors/Transforms/CheckConstraintsTransform.cpp index 88f02a3926f..3a6595ea4fb 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.cpp +++ b/src/Processors/Transforms/CheckConstraintsTransform.cpp @@ -73,7 +73,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) "Constraint expression returns nullable column that contains null value", backQuote(constraint_ptr->name), table_id.getNameForLogs(), - serializeAST(*(constraint_ptr->expr), true)); + serializeAST(*(constraint_ptr->expr))); result_column = nested_column; } @@ -116,7 +116,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1, - serializeAST(*(constraint_ptr->expr), true), + serializeAST(*(constraint_ptr->expr)), column_values_msg); } } diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index db37ac7c4c3..249ed8be428 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -35,7 +35,7 @@ String ConstraintsDescription::toString() const for (const auto & constraint : constraints) list.children.push_back(constraint); - return serializeAST(list, true); + return serializeAST(list); } ConstraintsDescription ConstraintsDescription::parse(const String & str) diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index c7aeaf8e4ef..06518a52c61 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -151,7 +151,7 @@ String IndicesDescription::toString() const for (const auto & index : *this) list.children.push_back(index.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..aecf0ac6d00 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -324,7 +324,7 @@ String ProjectionsDescription::toString() const for (const auto & projection : projections) list.children.push_back(projection.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } ProjectionsDescription ProjectionsDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context) From a24bf14450bdb1dad881330ca168566bf7e1f82f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 10:44:44 +0000 Subject: [PATCH 540/871] Use correct ZXID --- src/Coordination/KeeperStateMachine.cpp | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..a89b608aa69 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -390,7 +390,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) /// maybe some logs were preprocessed with log idx larger than the snapshot idx /// we have to apply them to the new storage - storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx()); + storage->applyUncommittedState(*snapshot_deserialization_result.storage, snapshot_deserialization_result.storage->getZXID()); storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 51a09b676dc..39d9200f913 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -65,7 +65,7 @@ int main(int argc, char *argv[]) CoordinationSettingsPtr settings = std::make_shared(); KeeperContextPtr keeper_context = std::make_shared(true); keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2])); - keeper_context->setSnapshotDisk(std::make_shared("LogDisk", argv[1])); + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", argv[1])); auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); From 27921a5d8f4218a92dafb6fdc145bf3891710e3a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 10:48:39 +0000 Subject: [PATCH 541/871] Docs: Add another reason for integer promotion rules in ClickHouse --- docs/en/sql-reference/functions/arithmetic-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 64fae0e82f0..054c59d5778 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,7 +6,7 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. +The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. From e74acda53ec3a7f8a536eb56e4a939935d10f8e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 20 Jul 2023 12:54:42 +0200 Subject: [PATCH 542/871] PRQL integration (#50686) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added prql-lib * Add PRQL parser * Extend stateless tests * Add unit tests for `ParserPRQL` --------- Co-authored-by: Ubuntu Co-authored-by: Ubuntu Co-authored-by: Александр Нам <47687537+seshWCS@users.noreply.github.com> --- rust/CMakeLists.txt | 1 + rust/prql/CMakeLists.txt | 3 + rust/prql/Cargo.lock | 569 ++++++++++++++++++ rust/prql/Cargo.toml | 20 + rust/prql/include/prql.h | 18 + rust/prql/src/lib.rs | 56 ++ src/Client/ClientBase.cpp | 4 + src/Common/config.h.in | 1 + src/Core/SettingsEnums.cpp | 4 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/executeQuery.cpp | 7 +- src/Parsers/CMakeLists.txt | 4 + src/Parsers/PRQL/ParserPRQLQuery.cpp | 86 +++ src/Parsers/PRQL/ParserPRQLQuery.h | 27 + src/Parsers/tests/gtest_Parser.cpp | 20 + src/configure_config.cmake | 3 + .../queries/0_stateless/02766_prql.reference | 19 + tests/queries/0_stateless/02766_prql.sh | 58 ++ 18 files changed, 899 insertions(+), 2 deletions(-) create mode 100644 rust/prql/CMakeLists.txt create mode 100644 rust/prql/Cargo.lock create mode 100644 rust/prql/Cargo.toml create mode 100644 rust/prql/include/prql.h create mode 100644 rust/prql/src/lib.rs create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.cpp create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.h create mode 100644 tests/queries/0_stateless/02766_prql.reference create mode 100755 tests/queries/0_stateless/02766_prql.sh diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 6700ead9786..41451fe0a1e 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -88,3 +88,4 @@ endfunction() add_rust_subdirectory (BLAKE3) add_rust_subdirectory (skim) +add_rust_subdirectory (prql) diff --git a/rust/prql/CMakeLists.txt b/rust/prql/CMakeLists.txt new file mode 100644 index 00000000000..65109d19a81 --- /dev/null +++ b/rust/prql/CMakeLists.txt @@ -0,0 +1,3 @@ +clickhouse_import_crate(MANIFEST_PATH Cargo.toml) +target_include_directories(_ch_rust_prql INTERFACE include) +add_library(ch_rust::prql ALIAS _ch_rust_prql) diff --git a/rust/prql/Cargo.lock b/rust/prql/Cargo.lock new file mode 100644 index 00000000000..da94e4ca852 --- /dev/null +++ b/rust/prql/Cargo.lock @@ -0,0 +1,569 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "_ch_rust_prql" +version = "0.1.0" +dependencies = [ + "prql-compiler", + "serde_json", +] + +[[package]] +name = "addr2line" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + +[[package]] +name = "aho-corasick" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" +dependencies = [ + "memchr", +] + +[[package]] +name = "anyhow" +version = "1.0.71" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" +dependencies = [ + "backtrace", +] + +[[package]] +name = "ariadne" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +dependencies = [ + "unicode-width", + "yansi", +] + +[[package]] +name = "backtrace" +version = "0.3.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "cc" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chumsky" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" +dependencies = [ + "hashbrown 0.12.3", + "stacker", +] + +[[package]] +name = "csv" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "either" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" + +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "equivalent" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" + +[[package]] +name = "getrandom" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62b02a5381cc465bd3041d84623d0fa3b66738b52b8e2fc3bab8ad63ab032f4a" + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "libc" +version = "0.2.147" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" + +[[package]] +name = "log" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" + +[[package]] +name = "memchr" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "object" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" + +[[package]] +name = "proc-macro2" +version = "1.0.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prql-compiler" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +dependencies = [ + "anyhow", + "ariadne", + "chumsky", + "csv", + "enum-as-inner", + "itertools", + "lazy_static", + "log", + "once_cell", + "regex", + "semver", + "serde", + "serde_json", + "serde_yaml", + "sqlformat", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "psm" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" +dependencies = [ + "cc", +] + +[[package]] +name = "quote" +version = "1.0.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "regex" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustversion" +version = "1.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc31bd9b61a32c31f9650d18add92aa83a49ba979c143eefd27fe7177b05bd5f" + +[[package]] +name = "ryu" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe232bdf6be8c8de797b22184ee71118d63780ea42ac85b61d1baa6d3b782ae9" + +[[package]] +name = "semver" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" +dependencies = [ + "serde", +] + +[[package]] +name = "serde" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d01b7404f9d441d3ad40e6a636a7782c377d2abdbe4fa2440e2edcc2f4f10db8" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dd83d6dde2b6b2d466e14d9d1acce8816dedee94f735eac6395808b3483c6d6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.23", +] + +[[package]] +name = "serde_json" +version = "1.0.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "452e67b9c20c37fa79df53201dc03839651086ed9bbe92b3ca585ca9fdaa7d85" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sqlformat" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" +dependencies = [ + "itertools", + "nom", + "unicode_categories", +] + +[[package]] +name = "sqlparser" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +dependencies = [ + "log", + "serde", +] + +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "unicode-ident" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73" + +[[package]] +name = "unicode-width" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" + +[[package]] +name = "unicode_categories" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml new file mode 100644 index 00000000000..314d1b52391 --- /dev/null +++ b/rust/prql/Cargo.toml @@ -0,0 +1,20 @@ +[package] +name = "_ch_rust_prql" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +prql-compiler = "0.8.1" +serde_json = "1.0" + +[lib] +crate-type = ["staticlib"] + +[profile.release] +debug = true + +[profile.release-thinlto] +inherits = "release" +lto = true diff --git a/rust/prql/include/prql.h b/rust/prql/include/prql.h new file mode 100644 index 00000000000..29158d7f30d --- /dev/null +++ b/rust/prql/include/prql.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +extern "C" { + +/// Converts a PRQL query to an SQL query. +/// @param query is a pointer to the beginning of the PRQL query. +/// @param size is the size of the PRQL query. +/// @param out is a pointer to a uint8_t pointer which will be set to the beginning of the null terminated SQL query or the error message. +/// @param out_size is the size of the string pointed by `out`. +/// @returns zero in case of success, non-zero in case of failure. +int64_t prql_to_sql(const uint8_t * query, uint64_t size, uint8_t ** out, uint64_t * out_size); + +/// Frees the passed in pointer which's memory was allocated by Rust allocators previously. +void prql_free_pointer(uint8_t * ptr_to_free); + +} // extern "C" diff --git a/rust/prql/src/lib.rs b/rust/prql/src/lib.rs new file mode 100644 index 00000000000..fb71d62d527 --- /dev/null +++ b/rust/prql/src/lib.rs @@ -0,0 +1,56 @@ +use prql_compiler::sql::Dialect; +use prql_compiler::{Options, Target}; +use std::ffi::{c_char, CString}; +use std::slice; + +fn set_output(result: String, out: *mut *mut u8, out_size: *mut u64) { + assert!(!out_size.is_null()); + let out_size_ptr = unsafe { &mut *out_size }; + *out_size_ptr = (result.len() + 1).try_into().unwrap(); + + assert!(!out.is_null()); + let out_ptr = unsafe { &mut *out }; + *out_ptr = CString::new(result).unwrap().into_raw() as *mut u8; +} + +#[no_mangle] +pub unsafe extern "C" fn prql_to_sql( + query: *const u8, + size: u64, + out: *mut *mut u8, + out_size: *mut u64, +) -> i64 { + let query_vec = unsafe { slice::from_raw_parts(query, size.try_into().unwrap()) }.to_vec(); + let maybe_prql_query = String::from_utf8(query_vec); + if maybe_prql_query.is_err() { + set_output( + String::from("The PRQL query must be UTF-8 encoded!"), + out, + out_size, + ); + return 1; + } + let prql_query = maybe_prql_query.unwrap(); + let opts = &Options { + format: true, + target: Target::Sql(Some(Dialect::ClickHouse)), + signature_comment: false, + color: false, + }; + let (is_err, res) = match prql_compiler::compile(&prql_query, &opts) { + Ok(sql_str) => (false, sql_str), + Err(err) => (true, err.to_string()), + }; + + set_output(res, out, out_size); + + match is_err { + true => 1, + false => 0, + } +} + +#[no_mangle] +pub unsafe extern "C" fn prql_free_pointer(ptr_to_free: *mut u8) { + std::mem::drop(CString::from_raw(ptr_to_free as *mut c_char)); +} diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 509dfe2e232..f5390037e6b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include @@ -72,6 +73,7 @@ #include #include #include +#include #include #include "config_version.h" @@ -338,6 +340,8 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu if (dialect == Dialect::kusto) parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); + else if (dialect == Dialect::prql) + parser = std::make_unique(max_length, settings.max_parser_depth); else parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1cb13d3ae3e..a2c18fc330f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -54,6 +54,7 @@ #cmakedefine01 USE_BORINGSSL #cmakedefine01 USE_BLAKE3 #cmakedefine01 USE_SKIM +#cmakedefine01 USE_PRQL #cmakedefine01 USE_OPENSSL_INTREE #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 1e2cbce9309..86400954e2f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -138,7 +138,9 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, - {"kusto", Dialect::kusto}}) + {"kusto", Dialect::kusto}, + {"kusto", Dialect::kusto}, + {"prql", Dialect::prql}}) // FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely? IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index c2783447441..c61afbd2bbf 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -207,6 +207,7 @@ enum class Dialect clickhouse, kusto, kusto_auto, + prql, }; DECLARE_SETTING_ENUM(Dialect) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b76d20f31d..66bc0bcb757 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -75,6 +75,7 @@ #include #include +#include namespace ProfileEvents { @@ -702,10 +703,14 @@ static std::tuple executeQueryImpl( /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } + else if (settings.dialect == Dialect::prql && !internal) + { + ParserPRQLQuery parser(max_query_size, settings.max_parser_depth); + ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + } else { ParserQuery parser(end, settings.allow_settings_after_format_in_insert); - /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index d5cf2bd4784..d74137f8a91 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -4,8 +4,12 @@ add_headers_and_sources(clickhouse_parsers .) add_headers_and_sources(clickhouse_parsers ./Access) add_headers_and_sources(clickhouse_parsers ./MySQL) add_headers_and_sources(clickhouse_parsers ./Kusto) +add_headers_and_sources(clickhouse_parsers ./PRQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils) +if (TARGET ch_rust::prql) + target_link_libraries(clickhouse_parsers PRIVATE ch_rust::prql) +endif () if (USE_DEBUG_HELPERS) # CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc. diff --git a/src/Parsers/PRQL/ParserPRQLQuery.cpp b/src/Parsers/PRQL/ParserPRQLQuery.cpp new file mode 100644 index 00000000000..b3733b727dc --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.cpp @@ -0,0 +1,86 @@ +#include +#include + +#include "Parsers/Lexer.h" +#include "config.h" + +#if USE_PRQL +# include +#endif + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int SUPPORT_IS_DISABLED; +} + +bool ParserPRQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserSetQuery set_p; + + if (set_p.parse(pos, node, expected)) + return true; + +#if !USE_PRQL + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "PRQL is not available. Rust code or PRQL itself may be disabled. Use another dialect!"); +#else + const auto * begin = pos->begin; + + // The same parsers are used in the client and the server, so the parser have to detect the end of a single query in case of multiquery queries + while (!pos->isEnd() && pos->type != TokenType::Semicolon) + ++pos; + + const auto * end = pos->begin; + + uint8_t * sql_query_ptr{nullptr}; + uint64_t sql_query_size{0}; + + const auto res + = prql_to_sql(reinterpret_cast(begin), static_cast(end - begin), &sql_query_ptr, &sql_query_size); + + SCOPE_EXIT({ prql_free_pointer(sql_query_ptr); }); + + const auto * sql_query_char_ptr = reinterpret_cast(sql_query_ptr); + const auto * const original_sql_query_ptr = sql_query_char_ptr; + + if (res != 0) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "PRQL syntax error: '{}'", sql_query_char_ptr); + } + chassert(sql_query_size > 0); + + ParserQuery query_p(end, false); + String error_message; + node = tryParseQuery( + query_p, + sql_query_char_ptr, + sql_query_char_ptr + sql_query_size - 1, + error_message, + false, + "", + false, + max_query_size, + max_parser_depth); + + if (!node) + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Error while parsing the SQL query generated from PRQL query :'{}'.\nPRQL Query:'{}'\nSQL query: '{}'", + error_message, + std::string_view{begin, end}, + std::string_view(original_sql_query_ptr, original_sql_query_ptr + sql_query_size)); + + + return true; +#endif +} +} diff --git a/src/Parsers/PRQL/ParserPRQLQuery.h b/src/Parsers/PRQL/ParserPRQLQuery.h new file mode 100644 index 00000000000..4fc450df6b6 --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace DB +{ +// Even when PRQL is disabled, it is not possible to exclude this parser because changing the dialect via `SET dialect = '...'` queries should succeed. +// Another solution would be disabling setting the dialect to PRQL, but it requires a lot of finicky conditional compiling around the Dialect setting enum. +// Therefore the decision, for now, is to use this parser even when PRQL is disabled to enable users to switch to another dialect. +class ParserPRQLQuery final : public IParserBase +{ +private: + // These fields are not used when PRQL is disabled at build time. + [[maybe_unused]] size_t max_query_size; + [[maybe_unused]] size_t max_parser_depth; + +public: + ParserPRQLQuery(size_t max_query_size_, size_t max_parser_depth_) : max_query_size{max_query_size_}, max_parser_depth{max_parser_depth_} + { + } + + const char * getName() const override { return "PRQL Statement"; } + +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..ef4ef05e35e 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -476,3 +477,22 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE 'pet%')" } }))); + +static constexpr size_t kDummyMaxQuerySize = 256 * 1024; +static constexpr size_t kDummyMaxParserDepth = 256; + +INSTANTIATE_TEST_SUITE_P( + ParserPRQL, + ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared(kDummyMaxQuerySize, kDummyMaxParserDepth)), + ::testing::ValuesIn(std::initializer_list{ + { + "from albums\ngroup [author_id] (\n aggregate [first_pushlied = min published]\n)\njoin a=author side:left [==author_id]\njoin p=purchases side:right [==author_id]\ngroup [a.id, p.purchase_id] (\n aggregate [avg_sell = min first_pushlied]\n)", + "WITH table_1 AS\n (\n SELECT\n MIN(published) AS _expr_0,\n author_id\n FROM albums\n GROUP BY author_id\n )\nSELECT\n a.id,\n p.purchase_id,\n MIN(table_0._expr_0) AS avg_sell\nFROM table_1 AS table_0\nLEFT JOIN author AS a ON table_0.author_id = a.author_id\nRIGHT JOIN purchases AS p ON table_0.author_id = p.author_id\nGROUP BY\n a.id,\n p.purchase_id", + }, + { + "from matches\nfilter start_date > @2023-05-30 # Some comment here\nderive [\n some_derived_value_1 = a + (b ?? 0), # And there\n some_derived_value_2 = c + some_derived_value\n]\nfilter some_derived_value_2 > 0\ngroup [country, city] (\n aggregate [\n average some_derived_value_2,\n aggr = max some_derived_value_2,\n ]\n)\nderive place = f\"{city} in {country}\"\nderive country_code = s\"LEFT(country, 2)\"\nsort [aggr, -country]\ntake 1..20", + "WITH\n table_3 AS\n (\n SELECT\n country,\n city,\n c + some_derived_value AS _expr_1\n FROM matches\n WHERE start_date > toDate('2023-05-30')\n ),\n table_1 AS\n (\n SELECT\n country,\n city,\n AVG(_expr_1) AS _expr_0,\n MAX(_expr_1) AS aggr\n FROM table_3 AS table_2\n WHERE _expr_1 > 0\n GROUP BY\n country,\n city\n )\nSELECT\n country,\n city,\n _expr_0,\n aggr,\n CONCAT(city, ' in ', country) AS place,\n LEFT(country, 2) AS country_code\nFROM table_1 AS table_0\nORDER BY\n aggr ASC,\n country DESC\nLIMIT 20", + }, + }))); diff --git a/src/configure_config.cmake b/src/configure_config.cmake index c11a19b36ea..ae6305705c2 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -25,6 +25,9 @@ endif() if (TARGET ch_rust::skim) set(USE_SKIM 1) endif() +if (TARGET ch_rust::prql) + set(USE_PRQL 1) +endif() if (TARGET OpenSSL::SSL) set(USE_SSL 1) endif() diff --git a/tests/queries/0_stateless/02766_prql.reference b/tests/queries/0_stateless/02766_prql.reference new file mode 100644 index 00000000000..90e0b26cee6 --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.reference @@ -0,0 +1,19 @@ +101 Hello, ClickHouse! 2 He +101 Granules are the smallest chunks of data read 2 Gr +102 Insert a lot of rows per batch 2 In +102 Sort your data based on your commonly-used queries 2 So +103 This is an awesome message 2 Th +103 42 +102 4.132209897041321 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 diff --git a/tests/queries/0_stateless/02766_prql.sh b/tests/queries/0_stateless/02766_prql.sh new file mode 100755 index 00000000000..f8bbd72af4e --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " +CREATE TEMPORARY TABLE IF NOT EXISTS aboba +( + user_id UInt32, + message String, + creation_date DateTime64, + metric Float32 +) +ENGINE = MergeTree +ORDER BY user_id; + +INSERT INTO aboba (user_id, message, creation_date, metric) VALUES (101, 'Hello, ClickHouse!', toDateTime('2019-01-01 00:00:00', 3, 'Europe/Amsterdam'), -1.0), (102, 'Insert a lot of rows per batch', toDateTime('2019-02-01 00:00:00', 3, 'Europe/Amsterdam'), 1.41421 ), (102, 'Sort your data based on your commonly-used queries', toDateTime('2019-03-01 00:00:00', 3, 'Europe/Amsterdam'), 2.718), (101, 'Granules are the smallest chunks of data read', toDateTime('2019-05-01 00:00:00', 3, 'Europe/Amsterdam'), 3.14159), (103, 'This is an awesome message', toDateTime('2019-04-01 00:00:00', 3, 'Europe/Amsterdam'), 42); + +SET dialect = 'prql'; + +from aboba +derive [ + a = 2, + b = s\"LEFT(message, 2)\" +] +select [ user_id, message, a, b ]; + +from aboba +filter user_id > 101 +group user_id ( + aggregate [ + metrics = sum metric + ] +); + +SET dialect = 'clickhouse'; + +SELECT '---'; +SELECT + user_id, + message, + toTimeZone(creation_date, 'Europe/Amsterdam') as creation_date, + metric +FROM aboba; +SELECT '---'; + +SET dialect = 'prql'; + +from aboba +select [ user_id, message, metric ] +derive creation_date = s\"toTimeZone(creation_date, 'Europe/Amsterdam')\" +select [ user_id, message, creation_date, metric]; + +from s\"SELECT * FROM system.users\" | select non_existent_column; # {serverError UNKNOWN_IDENTIFIER} +from non_existent_table; # {serverError UNKNOWN_TABLE} +" \ No newline at end of file From 84f6a7336c2d7ac547ad7030c389d4961f4ab8e4 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:03:42 +0800 Subject: [PATCH 543/871] Prevent going beyond the index of const_columns_to_remove. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 744d035d0ee..baf898481ab 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,11 +38,12 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); + size_t num_columns = const_columns_to_remove.size(); for (const auto & column_description : description_sorted_) { auto pos = header.getPositionByName(column_description.column_name); - if (!const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From cbcd48979cab1a3a4a0f0d5baaf8077164887cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Jul 2023 13:04:43 +0200 Subject: [PATCH 544/871] 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 d1c1a1d200f..774c3f223a6 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 f2d184cf1b002d18be152880ee2d82e57fed3b26 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:11:08 +0800 Subject: [PATCH 545/871] Consistent style for if statements. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index baf898481ab..63a9c3924a2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -43,9 +43,8 @@ FinishSortingTransform::FinishSortingTransform( { auto pos = header.getPositionByName(column_description.column_name); - if (pos < num_columns && !const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]) description_sorted_without_constants.push_back(column_description); - } } /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. From db1b53d1bb8ed6aa71f47010c81a7f3ebb0ae65d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:18:48 +0300 Subject: [PATCH 546/871] Update 01606_git_import.sh --- tests/queries/0_stateless/01606_git_import.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index c9aa2c7d82e..48558d79f93 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -13,7 +13,7 @@ cd $CLICKHOUSE_TMP || exit # Protection for network errors for _ in {1..10}; do rm -rf ./clickhouse-odbc - git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break + git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc 2> /dev/null > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break sleep 1 done From 2b29e3dc83d9ed6747acb4a249c9e1aca9616f21 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:22:22 +0300 Subject: [PATCH 547/871] Update MergeTreeBackgroundExecutor.cpp (#52261) --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 6eab4337162..e497a799274 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -269,7 +269,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) try { ALLOW_ALLOCATIONS_IN_SCOPE; - item->task->getQueryId(); + query_id = item->task->getQueryId(); need_execute_again = item->task->executeStep(); } catch (...) From f53ff5d4f2228b7016af5742ea1ae8f70ef772df Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:51:01 +0300 Subject: [PATCH 548/871] more fair queue for drop table sync (#52276) --- src/Interpreters/DatabaseCatalog.cpp | 17 ++++++++++++++++- src/Interpreters/DatabaseCatalog.h | 1 + 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 23a67f4bc2f..0e2e30eefee 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -697,6 +697,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(&Poco::Logger::get("DatabaseCatalog")) + , first_async_drop_in_queue(tables_marked_dropped.end()) { } @@ -959,9 +960,17 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr std::lock_guard lock(tables_marked_dropped_mutex); if (ignore_delay) - tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, drop_time}); + { + /// Insert it before first_async_drop_in_queue, so sync drop queries will have priority over async ones, + /// but the queue will remain fair for multiple sync drop queries. + tables_marked_dropped.emplace(first_async_drop_in_queue, TableMarkedAsDropped{table_id, table, dropped_metadata_path, drop_time}); + } else + { tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec}); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; + } tables_marked_dropped_ids.insert(table_id.uuid); CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1); @@ -1012,6 +1021,8 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) /// This maybe throw exception. renameNoReplace(latest_metadata_dropped_path, table_metadata_path); + if (first_async_drop_in_queue == it_dropped_table) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it_dropped_table); [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table.table_id.uuid); assert(removed); @@ -1074,6 +1085,8 @@ void DatabaseCatalog::dropTableDataTask() table = std::move(*it); LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {}", tables_marked_dropped.size(), tables_in_use_count, table.table_id.getNameForLogs()); + if (first_async_drop_in_queue == it) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it); /// Schedule the task as soon as possible, while there are suitable tables to drop. schedule_after_ms = 0; @@ -1110,6 +1123,8 @@ void DatabaseCatalog::dropTableDataTask() table.drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; std::lock_guard lock(tables_marked_dropped_mutex); tables_marked_dropped.emplace_back(std::move(table)); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; /// If list of dropped tables was empty, schedule a task to retry deletion. if (tables_marked_dropped.size() == 1) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d502505027f..805d7786569 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -323,6 +323,7 @@ private: mutable std::mutex ddl_guards_mutex; TablesMarkedAsDropped tables_marked_dropped TSA_GUARDED_BY(tables_marked_dropped_mutex); + TablesMarkedAsDropped::iterator first_async_drop_in_queue TSA_GUARDED_BY(tables_marked_dropped_mutex); std::unordered_set tables_marked_dropped_ids TSA_GUARDED_BY(tables_marked_dropped_mutex); mutable std::mutex tables_marked_dropped_mutex; From fc3be766f7347315a972a9854e25ac72d63dd0e0 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Thu, 20 Jul 2023 19:58:51 +0800 Subject: [PATCH 549/871] ignore test --- .../02813_system_events_and_metrics_add_alias.reference | 4 ---- .../0_stateless/02813_system_events_and_metrics_add_alias.sql | 4 ---- 2 files changed, 8 deletions(-) delete mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference delete mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference deleted file mode 100644 index 93be2764a57..00000000000 --- a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String,\n `name` String\n)\nENGINE = SystemMetrics\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -1 -CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String,\n `name` String\n)\nENGINE = SystemEvents\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -1 diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql deleted file mode 100644 index a2250608f8f..00000000000 --- a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql +++ /dev/null @@ -1,4 +0,0 @@ -show create table system.metrics; -select equals((select count() from system.metrics where name=metric) as r1, (select count() from system.metrics) as r2); -show create table system.events; -select equals((select count() from system.events where name=event) as r1, (select count() from system.events) as r2); \ No newline at end of file From d16d4449432999cdee3393b1f47b4a7d7c5314a6 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 20 Jul 2023 12:24:52 +0200 Subject: [PATCH 550/871] MaterializedMySQL: Add support of double quoted comments --- src/Parsers/ExpressionElementParsers.cpp | 33 +++++++++++++++++ src/Parsers/ExpressionElementParsers.h | 15 ++++++++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 2 +- .../materialized_with_ddl.py | 35 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 5 files changed, 90 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3a7e8790bb4..0149526da79 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1900,6 +1900,39 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } +bool ParserMySQLComment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (pos->type != TokenType::QuotedIdentifier && pos->type != TokenType::StringLiteral) + return false; + String s; + ReadBufferFromMemory in(pos->begin, pos->size()); + try + { + if (pos->type == TokenType::StringLiteral) + readQuotedStringWithSQLStyle(s, in); + else + readDoubleQuotedStringWithSQLStyle(s, in); + } + catch (const Exception &) + { + expected.add(pos, "string literal or double quoted string"); + return false; + } + + if (in.count() != pos->size()) + { + expected.add(pos, "string literal or double quoted string"); + return false; + } + + auto literal = std::make_shared(s); + literal->begin = pos; + literal->end = ++pos; + node = literal; + return true; +} + + bool ParserMySQLGlobalVariable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (pos->type != TokenType::DoubleAt) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index cc88faf2653..f33f2d99f71 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -367,6 +367,21 @@ protected: }; +/** MySQL comment: + * CREATE TABLE t ( + * i INT PRIMARY KEY, + * first_name VARCHAR(255) COMMENT 'FIRST_NAME', + * last_name VARCHAR(255) COMMENT "LAST_NAME" + * ) + */ +class ParserMySQLComment : public IParserBase +{ +protected: + const char * getName() const override { return "MySQL comment parser"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** MySQL-style global variable: @@var */ class ParserMySQLGlobalVariable : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index e585dcb670c..e5f2b7870e2 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -50,7 +50,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), OptionDescribe("UNIQUE", "unique_key", std::make_unique()), OptionDescribe("KEY", "primary_key", std::make_unique()), - OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), OptionDescribe("CHARSET", "charset", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), 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 8cf9e67bf63..f7a930ec00b 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1617,6 +1617,41 @@ def materialized_with_column_comments_test(clickhouse_node, mysql_node, service_ mysql_node.query("DROP DATABASE materialized_with_column_comments_test") +def double_quoted_comment(clickhouse_node, mysql_node, service_name): + db = "comment_db" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + mysql_node.query( + f'CREATE TABLE {db}.t1 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT "ID")' + ) + mysql_node.query( + f"CREATE TABLE {db}.t2 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT 'ID')" + ) + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\nt2\n", + ) + + # incremental + mysql_node.query( + f'CREATE TABLE {db}.t3 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT "ID")' + ) + mysql_node.query( + f"CREATE TABLE {db}.t4 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT 'ID')" + ) + check_query( + clickhouse_node, f"SHOW TABLES FROM {db} FORMAT TSV", "t1\nt2\nt3\nt4\n" + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + def materialized_with_enum8_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS materialized_with_enum8_test") clickhouse_node.query("DROP DATABASE IF EXISTS materialized_with_enum8_test") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..0166f7d1d33 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -416,6 +416,12 @@ def test_materialized_with_column_comments( ) +def test_double_quoted_comment(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.double_quoted_comment( + clickhouse_node, started_mysql_8_0, "mysql80" + ) + + def test_materialized_with_enum( started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node ): From fe934d3059936cd203952cfe5881ff7243001ae9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:38:41 +0000 Subject: [PATCH 551/871] 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 9f2bf177c96..f556df0a088 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 db5d1a2f5d9..d138b07d3ae 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 96f36f03949..677ed011960 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 ffa72d841be..5dc40494115 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 4cfefbc5527..0c915f54cff 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 552/871] Better --- base/poco/Foundation/include/Poco/URI.h | 6 +++-- base/poco/Foundation/src/URI.cpp | 34 +++++++++++++------------ 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index 5e6e7efd938..f4505147ced 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: 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 @@ protected: private: void encodePath(std::string & encodedStr) const; + void decodePath(const std::string & encodedStr); + std::string _scheme; std::string _userInfo; @@ -360,7 +362,7 @@ private: 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 9bad1b39a87..3354c69d188 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) - encode(_path, RESERVED_PATH, encodedStr); - else + 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 + 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 3c9e46b557a882085fdcdce5d74ad12329457db3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 16:19:12 +0300 Subject: [PATCH 553/871] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 43 ++++++++++++++++++++++++++---- 1 file changed, 38 insertions(+), 5 deletions(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 6e694b4fdbd..0fb12e89ce9 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -26,10 +26,11 @@ else: DRY_RUN_MARK = "" -MAX_FAILURES_DEFAULT = 40 +MAX_FAILURES_DEFAULT = 30 SLACK_URL_DEFAULT = DRY_RUN_MARK -FLAKY_ALERT_PROBABILITY = 0.20 +FLAKY_ALERT_PROBABILITY = 0.50 +REPORT_NO_FAILURES_PROBABILITY = 0.99 MAX_TESTS_TO_REPORT = 4 @@ -89,6 +90,22 @@ WHERE 1 AND check_name ILIKE check_name_pattern """ +# Returns percentage of failed checks (once per day, at noon) +FAILED_CHECKS_PERCENTAGE_QUERY = """ +SELECT if(toHour(now('Europe/Amsterdam')) = 12, v, 0) +FROM +( + SELECT + countDistinctIf((commit_sha, check_name), (test_status LIKE 'F%') AND (check_status != 'success')) + / countDistinct((commit_sha, check_name)) AS v + FROM checks + WHERE 1 + AND (pull_request_number = 0) + AND (test_status != 'SKIPPED') + AND (check_start_time > (now() - toIntervalDay(1))) +) +""" + # It shows all recent failures of the specified test (helps to find when it started) ALL_RECENT_FAILURES_QUERY = """ WITH @@ -202,9 +219,9 @@ def get_too_many_failures_message_impl(failures_count): curr_failures = int(failures_count[0][0]) prev_failures = int(failures_count[0][1]) if curr_failures == 0 and prev_failures != 0: - return ( - "Looks like CI is completely broken: there are *no failures* at all... 0_o" - ) + if random.random() < REPORT_NO_FAILURES_PROBABILITY: + return None + return "Wow, there are *no failures* at all... 0_o" if curr_failures < MAX_FAILURES: return None if prev_failures < MAX_FAILURES: @@ -227,6 +244,19 @@ def get_too_many_failures_message(failures_count): return msg +def get_failed_checks_percentage_message(percentage): + p = percentage[0][0] * 100 + + # Always report more than 1% of failed checks + # For <= 1%: higher percentage of failures == higher probability + if p <= random.random(): + return None + + msg = ":alert: " if p > 1 else "Only " if p < 0.5 else "" + msg += "*{0:.2f}%* of all checks in master have failed yesterday".format(p) + return msg + + def split_slack_message(long_message): lines = long_message.split("\n") messages = [] @@ -280,6 +310,9 @@ def query_and_alert_if_needed(query, get_message_func): def check_and_alert(): query_and_alert_if_needed(NEW_BROKEN_TESTS_QUERY, get_new_broken_tests_message) query_and_alert_if_needed(COUNT_FAILURES_QUERY, get_too_many_failures_message) + query_and_alert_if_needed( + FAILED_CHECKS_PERCENTAGE_QUERY, get_failed_checks_percentage_message + ) def lambda_handler(event, context): From 8649c84461f3c27bdf9fcab4db1884b21603dc2e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 13:28:37 +0000 Subject: [PATCH 554/871] 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 4394083a1c3..80c3535ef4e 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 e8daec9e164..5514c34f4ef 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 f997adfe27e1bd3bb772857fb11fae962c373b9e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 14:02:55 +0000 Subject: [PATCH 555/871] Retry if sessions not closed because missing leader --- src/Coordination/KeeperDispatcher.cpp | 37 +++++++++------- tests/integration/test_keeper_session/test.py | 42 ++++++++++++++----- 2 files changed, 54 insertions(+), 25 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..dfb621eb0ad 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -473,23 +473,30 @@ void KeeperDispatcher::shutdown() session_to_response_callback.clear(); } - // if there is no leader, there is no reason to do CLOSE because it's a write request - if (server && hasLeader() && !close_requests.empty()) + if (server && !close_requests.empty()) { - LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); - const auto raft_result = server->putRequestBatch(close_requests); - auto sessions_closing_done_promise = std::make_shared>(); - auto sessions_closing_done = sessions_closing_done_promise->get_future(); - raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( - nuraft::cmd_result> & /*result*/, - nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); + // if there is no leader, there is no reason to do CLOSE because it's a write request + if (hasLeader()) + { + LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); + const auto raft_result = server->putRequestBatch(close_requests); + auto sessions_closing_done_promise = std::make_shared>(); + auto sessions_closing_done = sessions_closing_done_promise->get_future(); + raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( + nuraft::cmd_result> & /*result*/, + nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); - auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); - if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) - LOG_WARNING( - log, - "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", - session_shutdown_timeout); + auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); + if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) + LOG_WARNING( + log, + "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", + session_shutdown_timeout); + } + else + { + LOG_INFO(log, "Sessions cannot be closed during shutdown because there is no active leader"); + } } if (server) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index e57057a8258..68147865cd2 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -6,6 +6,7 @@ import socket import struct from kazoo.client import KazooClient +from kazoo.exceptions import NoNodeError # from kazoo.protocol.serialization import Connect, read_buffer, write_buffer @@ -162,17 +163,38 @@ def test_session_timeout(started_cluster): def test_session_close_shutdown(started_cluster): wait_nodes() - node1_zk = get_fake_zk(node1.name) - node2_zk = get_fake_zk(node2.name) + node1_zk = None + node2_zk = None + for i in range(20): + node1_zk = get_fake_zk(node1.name) + node2_zk = get_fake_zk(node2.name) - eph_node = "/test_node" - node2_zk.create(eph_node, ephemeral=True) - node1_zk.sync(eph_node) - assert node1_zk.exists(eph_node) != None + eph_node = "/test_node" + node2_zk.create(eph_node, ephemeral=True) + node1_zk.sync(eph_node) - # shutdown while session is active - node2.stop_clickhouse() + node1_zk.exists(eph_node) != None - assert node1_zk.exists(eph_node) == None + # restart while session is active so it's closed during shutdown + node2.restart_clickhouse() - node2.start_clickhouse() + if node1_zk.exists(eph_node) == None: + break + + assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + + try: + node1_zk.delete(eph_node) + except NoNodeError: + pass + + assert node1_zk.exists(eph_node) == None + + destroy_zk_client(node1_zk) + node1_zk = None + destroy_zk_client(node2_zk) + node2_zk = None + + time.sleep(1) + else: + assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file From 5decb1f5c555d2465724f9bc3c555c157f9deb81 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 14:11:11 +0000 Subject: [PATCH 556/871] Automatic style fix --- tests/integration/test_keeper_session/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index 68147865cd2..cd012ad6e9e 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -181,7 +181,9 @@ def test_session_close_shutdown(started_cluster): if node1_zk.exists(eph_node) == None: break - assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + assert node2.contains_in_log( + "Sessions cannot be closed during shutdown because there is no active leader" + ) try: node1_zk.delete(eph_node) @@ -197,4 +199,4 @@ def test_session_close_shutdown(started_cluster): time.sleep(1) else: - assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file + assert False, "Session wasn't properly cleaned up on shutdown" From 046bf55dc084d4df91ecfddb8e22aa6f9300fa43 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 14:17:33 +0000 Subject: [PATCH 557/871] Incorporate feedback --- .../functions/arithmetic-functions.md | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 054c59d5778..69f1816b7df 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,9 +6,20 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. +Arithmetic functions work for any two operands of type `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64`. -The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. +Before performing the operation, both operands are casted to the result type. The result type is determined as follows (unless specified +differently in the function documentation below): +- If both operands are up to 32 bits wide, the size of the result type will be the size of the next bigger type following the bigger of the + two operands (integer size promotion). For example, `UInt8 + UInt16 = UInt32` or `Float32 * Float32 = Float64`. +- If one of the operands has 64 or more bits, the size of the result type will be the same size as the bigger of the two operands. For + example, `UInt32 + UInt128 = UInt128` or `Float32 * Float64 = Float64`. +- If one of the operands is signed, the result type will also be signed, otherwise it will be signed. For example, `UInt32 * Int32 = Int64`. + +These rules make sure that the result type will be the smallest type which can represent all possible results. While this introduces a risk +of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of +64 bit. This behavior also guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer +type. Example: @@ -22,8 +33,6 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 └───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ ``` -Arithmetic functions work for any pair of `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64` values. - Overflows are produced the same way as in C++. ## plus @@ -68,7 +77,7 @@ Alias: `a \* b` (operator) ## divide -Calculates the quotient of two values `a` and `b`. The result is always a floating-point value. If you need integer division, you can use the `intDiv` function. +Calculates the quotient of two values `a` and `b`. The result type is always [Float64](../../sql-reference/data-types/float.md). Integer division is provided by the `intDiv` function. Division by 0 returns `inf`, `-inf`, or `nan`. @@ -84,7 +93,7 @@ Alias: `a / b` (operator) Performs an integer division of two values `a` by `b`, i.e. computes the quotient rounded down to the next smallest integer. -The result has the same type as the dividend (the first parameter). +The result has the same width as the dividend (the first parameter). An exception is thrown when dividing by zero, when the quotient does not fit in the range of the dividend, or when dividing a minimal negative number by minus one. @@ -135,7 +144,7 @@ intDivOrZero(a, b) Calculates the remainder of the division of two values `a` by `b`. -The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result is a floating-point number. +The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result type is [Float64](../../sql-reference/data-types/float.md). The remainder is computed like in C++. Truncated division is used for negative numbers. From 8adf57a6981610936acc84f3c69342682952ff0a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 14:18:32 +0000 Subject: [PATCH 558/871] 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 542e173deb9..2d8c5a5f61f 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 e86f7fa9b39..e0fbd4b2948 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 e467264588a6435199879fd89d1dc995c9e37c63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:56:30 +0300 Subject: [PATCH 559/871] Update src/IO/HTTPCommon.cpp --- src/IO/HTTPCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index a5816911c09..ddd7ccbe483 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -81,7 +81,7 @@ namespace Session::close(); LOG_TRACE( log, - "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", + "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", Session::getResolvedHost(), Session::getHost(), Session::getPort()); From c0aa3e456705e3ef75ed09683f4e9ed6d9151917 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:59:43 +0300 Subject: [PATCH 560/871] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 0fb12e89ce9..ea883e3cda3 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -245,7 +245,7 @@ def get_too_many_failures_message(failures_count): def get_failed_checks_percentage_message(percentage): - p = percentage[0][0] * 100 + p = float(percentage[0][0]) * 100 # Always report more than 1% of failed checks # For <= 1%: higher percentage of failures == higher probability From ea252e2f612afd9e83c1aa000af945eebbe18a16 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 15:05:07 +0000 Subject: [PATCH 561/871] Disable analyzer setting in backward_compatibility integration tests. --- tests/integration/helpers/cluster.py | 6 +++++- tests/integration/test_backward_compatibility/test.py | 2 ++ .../test_aggregate_fixed_key.py | 5 +++-- .../test_aggregate_function_state.py | 6 ++++-- .../test_backward_compatibility/test_convert_ordinary.py | 1 + .../test_backward_compatibility/test_cte_distributed.py | 7 ++++--- .../test_data_skipping_indices.py | 1 + .../test_backward_compatibility/test_functions.py | 3 ++- .../test_in_memory_parts_still_read.py | 1 + .../test_insert_profile_events.py | 3 ++- .../test_ip_types_binary_compatibility.py | 1 + .../test_memory_bound_aggregation.py | 4 +++- .../test_normalized_count_comparison.py | 3 ++- .../test_select_aggregate_alias_column.py | 3 ++- .../test_short_strings_aggregation.py | 4 +++- .../test_vertical_merges_from_compact_parts.py | 2 ++ 16 files changed, 38 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c52442ecb9c..c85fbb8ad9e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1533,6 +1533,7 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, + allow_analyzer=True, hostname=None, env_variables=None, image="clickhouse/integration-test", @@ -1630,6 +1631,7 @@ class ClickHouseCluster: with_hive=with_hive, with_coredns=with_coredns, with_cassandra=with_cassandra, + allow_analyzer=allow_analyzer, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -3169,6 +3171,7 @@ class ClickHouseInstance: with_hive, with_coredns, with_cassandra, + allow_analyzer, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, @@ -3256,6 +3259,7 @@ class ClickHouseInstance: self.with_hive = with_hive self.with_coredns = with_coredns self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) + self.allow_analyzer = allow_analyzer self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -4227,7 +4231,7 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None: + if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index ea1d3ab9c07..c3d3b8aad34 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,11 +10,13 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, + allow_analyzer=False ) node2 = cluster.add_instance( "node2", main_configs=["configs/wide_parts_only.xml", "configs/no_compress_marks.xml"], with_zookeeper=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 01c9736c354..cf258987cbf 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -9,9 +9,10 @@ node1 = cluster.add_instance( image="yandex/clickhouse-server", tag="21.3", with_installed_binary=True, + allow_analyzer=False, ) -node2 = cluster.add_instance("node2", with_zookeeper=True) -node3 = cluster.add_instance("node3", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=True, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py index 1f6d405603a..3a936239cc8 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,9 +19,10 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) -node4 = cluster.add_instance("node4", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) +node4 = cluster.add_instance("node4", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 8b1afd358eb..36facdd59b1 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -9,6 +9,7 @@ node = cluster.add_instance( stay_alive=True, with_zookeeper=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index 7ea0d2d9f21..c68468aad75 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.3.14", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) @@ -31,7 +32,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range @@ -49,7 +50,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range diff --git a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py index c65dc6d3841..46ab27d2ab0 100644 --- a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py +++ b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="21.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index afb19901e74..fa24b146fec 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -upstream = cluster.add_instance("upstream") +upstream = cluster.add_instance("upstream", allow_analyzer=False) backward = cluster.add_instance( "backward", image="clickhouse/clickhouse-server", @@ -19,6 +19,7 @@ backward = cluster.add_instance( # Affected at least: singleValueOrNull, last_value, min, max, any, anyLast, anyHeavy, first_value, argMin, argMax tag="22.6", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py index d55f155918e..cd67f1f6344 100644 --- a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py +++ b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="23.4", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 0fd453e57d4..8564c6b5952 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -7,12 +7,13 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -upstream_node = cluster.add_instance("upstream_node") +upstream_node = cluster.add_instance("upstream_node", allow_analyzer=False) old_node = cluster.add_instance( "old_node", image="clickhouse/clickhouse-server", tag="22.5.1.2079", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py index bb40dff27ac..04016755a24 100644 --- a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py +++ b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py @@ -10,6 +10,7 @@ node_22_6 = cluster.add_instance( tag="22.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index d76c4eba409..96b41c81384 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index fcdedd29dad..3cd708d5029 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index 8bdae54a889..7e10b6ab430 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 17a7282b7b5..e4fda618031 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 3d006caad0d..82ffcc20b60 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -11,12 +11,14 @@ node_old = cluster.add_instance( stay_alive=True, with_installed_binary=True, with_zookeeper=True, + allow_analyzer=False, ) node_new = cluster.add_instance( "node2", main_configs=["configs/no_compress_marks.xml"], with_zookeeper=True, stay_alive=True, + allow_analyzer=False, ) From a26de1b370e8c09c548528ffbe3337cbf2340012 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 15:12:55 +0000 Subject: [PATCH 562/871] Automatic style fix --- tests/integration/helpers/cluster.py | 5 ++++- tests/integration/test_backward_compatibility/test.py | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c85fbb8ad9e..0ac2f330b1e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4231,7 +4231,10 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: + if ( + os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None + and self.allow_analyzer + ): write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index c3d3b8aad34..6f21b184a95 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, - allow_analyzer=False + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", From e6624a07e4fe938b55dd6bc5d8cbabd0ed93d2d7 Mon Sep 17 00:00:00 2001 From: AlexBykovski Date: Thu, 20 Jul 2023 18:54:48 +0300 Subject: [PATCH 563/871] Update build-osx.md syntax error in command for compiler for OSx compilation --- docs/ru/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/build-osx.md b/docs/ru/development/build-osx.md index 9a1f9c9347d..6b4e612b13f 100644 --- a/docs/ru/development/build-osx.md +++ b/docs/ru/development/build-osx.md @@ -68,7 +68,7 @@ $ /bin/bash -c "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/ $ rm -rf build $ mkdir build $ cd build - $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER==$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. + $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake --build . --config RelWithDebInfo $ cd .. From 97e54d6ebaa174f8d2ae291ddec20fd879b29bfa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:13:25 +0000 Subject: [PATCH 564/871] Fix test_backup_restore_on_cluster flakiness caused by missing replica syncs --- tests/integration/test_backup_restore_on_cluster/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 6af3a7dbab8..39496b8a5c8 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -580,6 +580,7 @@ def test_required_privileges(): node1.query( f"RESTORE TABLE tbl AS tbl2 ON CLUSTER 'cluster' FROM {backup_name}", user="u1" ) + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl2") assert node2.query("SELECT * FROM tbl2") == "100\n" @@ -593,6 +594,7 @@ def test_required_privileges(): node1.query("GRANT INSERT, CREATE TABLE ON tbl TO u1") node1.query(f"RESTORE ALL ON CLUSTER 'cluster' FROM {backup_name}", user="u1") + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") assert node2.query("SELECT * FROM tbl") == "100\n" From 13f8d72f54433a790f3efcb054db389e4fdd53f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jul 2023 17:46:22 +0200 Subject: [PATCH 565/871] Wait for zero copy replication lock even if some disks don't support it --- .../MergeTree/MergeFromLogEntryTask.cpp | 8 +++++-- .../MergeTree/MutateFromLogEntryTask.cpp | 6 ++++- .../ReplicatedMergeMutateTaskBase.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 22 +++++++++++++++---- 4 files changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 9f54c554c85..883cfee89c8 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -230,7 +230,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() /// the fast replica is not overloaded because amount of executing merges doesn't affect the ability to acquire locks for new merges. /// /// So here we trying to solve it with the simplest solution -- sleep random time up to 500ms for 1GB part and up to 7 seconds for 300GB part. - /// It can sound too much, but we are trying to aquite these locks in background tasks which can be scheduled each 5 seconds or so. + /// It can sound too much, but we are trying to acquire these locks in background tasks which can be scheduled each 5 seconds or so. double start_to_sleep_seconds = std::logf(storage_settings_ptr->zero_copy_merge_mutation_min_parts_size_sleep_before_lock.value); uint64_t right_border_to_sleep_ms = static_cast((std::log(estimated_space_for_merge) - start_to_sleep_seconds + 0.5) * 1000); uint64_t time_to_sleep_milliseconds = std::min(10000UL, std::uniform_int_distribution(1, 1 + right_border_to_sleep_ms)(rng)); @@ -245,7 +245,11 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { - LOG_DEBUG(log, "Merge of part {} started by some other replica, will wait it and fetch merged part", entry.new_part_name); + LOG_DEBUG( + log, + "Merge of part {} started by some other replica, will wait for it and fetch merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); /// Don't check for missing part -- it's missing because other replica still not /// finished merge. diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 6cb9d50436e..164b541d2b8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -154,8 +154,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { + LOG_DEBUG( + log, + "Mutation of part {} started by some other replica, will wait for it and mutated merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); - LOG_DEBUG(log, "Mutation of part {} started by some other replica, will wait it and mutated merged part", entry.new_part_name); return PrepareResult{ .prepared_successfully = false, diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index b4748ee77ea..6ad77119016 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -174,7 +174,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() part_log_writer = prepare_result.part_log_writer; - /// Avoid resheduling, execute fetch here, in the same thread. + /// Avoid rescheduling, execute fetch here, in the same thread. if (!prepare_result.prepared_successfully) return execute_fetch(prepare_result.need_to_check_missing_part_in_fetch); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 07f46c07466..3264de850a0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1370,13 +1370,27 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (data_settings->allow_remote_fs_zero_copy_replication) { auto disks = storage.getDisks(); - bool only_s3_storage = true; + DiskPtr disk_with_zero_copy = nullptr; for (const auto & disk : disks) - if (!disk->supportZeroCopyReplication()) - only_s3_storage = false; + { + if (disk->supportZeroCopyReplication()) + { + disk_with_zero_copy = disk; + break; + } + } + /// Technically speaking if there are more than one disk that could store the part (a local hot + cloud cold) + /// It would be possible for the merge to happen concurrently with other replica if the other replica is doing + /// a merge using zero-copy and the cloud storage, and the local replica uses the local storage instead + /// The question is, is it worth keep retrying to do the merge over and over for the opportunity to do + /// double the work? Probably not + /// So what we do is that, even if hot merge could happen, check the zero copy lock anyway. + /// Keep in mind that for the zero copy lock check to happen (via existing_zero_copy_locks) we need to + /// have failed first because of it and added it via watchZeroCopyLock. Considering we've already tried to + /// use cloud storage and zero-copy replication, the most likely scenario is that we'll try again String replica_to_execute_merge; - if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge)) + if (disk_with_zero_copy && storage.checkZeroCopyLockExists(entry.new_part_name, disk_with_zero_copy, replica_to_execute_merge)) { constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after."; out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge); From ed59870f92fa2893c9c105eaaeff82b1efaede22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Thu, 20 Jul 2023 18:04:58 +0200 Subject: [PATCH 566/871] Update LRUFileCachePriority.cpp --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 18862e154da..33e567b7a76 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -7,6 +7,7 @@ namespace CurrentMetrics { extern const Metric FilesystemCacheSize; + extern const Metric FilesystemCacheSizeLimit; extern const Metric FilesystemCacheElements; } @@ -101,6 +102,7 @@ void LRUFileCachePriority::updateSize(int64_t size) { current_size += size; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, getSizeLimit()); } void LRUFileCachePriority::updateElementsCount(int64_t num) From b3c42a1171e3f631e8985b80fc3c822c7ac87dd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Thu, 20 Jul 2023 18:06:54 +0200 Subject: [PATCH 567/871] Update CurrentMetrics.cpp with FilesystemCacheSizeLimit metric --- src/Common/CurrentMetrics.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 626b43aea2c..583b13cf79d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -187,6 +187,7 @@ M(CacheFileSegments, "Number of existing cache file segments") \ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ + M(FilesystemCacheSizeLimit, "Filesystem cache size limit in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ From 920887f315e108da3b385986dee329a28aed65fb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:43:59 +0000 Subject: [PATCH 568/871] Done --- .../test_replicated_merge_tree_encrypted_disk/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py index 05d7bbb7282..25d30eb9c82 100644 --- a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py +++ b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py @@ -67,6 +67,8 @@ def optimize_table(): def check_table(): expected = [[1, "str1"], [2, "str2"]] + node1.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") + node2.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node1.query("CHECK TABLE tbl") == "1\n" From 6b3a508a23e62d5459ad2a19a3bfc91ca96ccb8f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:52:45 +0000 Subject: [PATCH 569/871] Done --- tests/queries/0_stateless/02122_parallel_formatting.lib | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting.lib b/tests/queries/0_stateless/02122_parallel_formatting.lib index 56119012788..5175e004cc5 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting.lib +++ b/tests/queries/0_stateless/02122_parallel_formatting.lib @@ -11,14 +11,14 @@ non_parallel_file=$CLICKHOUSE_TMP/$CLICKHOUSE_TEST_UNIQUE_NAME"_non_parallel" format=$1 echo $format-1 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file echo $format-2 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file From 500f1e6757b721ecc8733f5e8bf41c765a631918 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 18:55:41 +0200 Subject: [PATCH 570/871] Follow up to #49698 --- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index adbc95a2cf2..f2923b60bfd 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int TOO_MANY_PARTS; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -591,11 +590,8 @@ void MaterializedPostgreSQLConsumer::syncTables() } catch (DB::Exception & e) { - if (e.code() == ErrorCodes::TOO_MANY_PARTS) - { - /// Retry this buffer later. - storage_data.buffer.columns = result_rows.mutateColumns(); - } + /// Retry this buffer later. + storage_data.buffer.columns = result_rows.mutateColumns(); throw; } From 045ecdf71f544cafb4f5c3eda20ee6f9d593f614 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 19:03:06 +0200 Subject: [PATCH 571/871] Update src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f2923b60bfd..d01746ddf1b 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -588,7 +588,7 @@ void MaterializedPostgreSQLConsumer::syncTables() executor.execute(); } } - catch (DB::Exception & e) + catch (...) { /// Retry this buffer later. storage_data.buffer.columns = result_rows.mutateColumns(); From 09e6bbc0e2ac634cde658b9c53e599d124d0a3d8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 21:10:59 +0300 Subject: [PATCH 572/871] Update DataPartsExchange.cpp --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 23bbc1c7f9d..6a3bf2940e9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -353,8 +353,14 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) { /// It is important to include Outdated parts here because remote replicas cannot reliably /// determine the local state of the part, so queries for the parts in these states are completely normal. - auto part = data.getPartIfExists( - name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + MergeTreeData::DataPartPtr part; + + /// Ephemeral zero-copy lock may be lost for PreActive parts + bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; + if (zero_copy_enabled) + part = data.getPartIfExists(name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + else + part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) return part; From 24371c33bfd5037455cb025b057fb413ee1be396 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 21 Jul 2023 09:24:16 +0800 Subject: [PATCH 573/871] remove DelayedBlocksTask::finish --- src/Processors/Transforms/JoiningTransform.cpp | 3 ++- src/Processors/Transforms/JoiningTransform.h | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 5480fea27a4..4e7868ea1c2 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -375,7 +375,8 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() input.setNotNeeded(); } - if (task->finished) + // When delayed_blocks is nullptr, it means that all buckets have been joined. + if (!task->delayed_blocks) { input.close(); output.finish(); diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 5e7403dbbdb..a308af03662 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -115,7 +115,7 @@ class DelayedBlocksTask : public ChunkInfo { public: - explicit DelayedBlocksTask() : finished(true) {} + DelayedBlocksTask() = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) @@ -125,7 +125,6 @@ public: IBlocksStreamPtr delayed_blocks = nullptr; JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; - bool finished = false; }; using DelayedBlocksTaskPtr = std::shared_ptr; From f0e277f94a642647cfd3eb5ebc722b486d9203b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 21 Jul 2023 06:45:35 +0200 Subject: [PATCH 574/871] Rename TaskStatsInfoGetter into NetlinkMetricsProvider There is ProcfsMetricsProvider, so by analogy to it. Signed-off-by: Azat Khuzhin --- src/Common/CurrentThread.cpp | 1 - ...oGetter.cpp => NetlinkMetricsProvider.cpp} | 22 +++++++++---------- ...sInfoGetter.h => NetlinkMetricsProvider.h} | 6 ++--- src/Common/ThreadProfileEvents.cpp | 6 ++--- src/Disks/IO/ThreadPoolReader.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/SynchronousReader.cpp | 2 +- 7 files changed, 20 insertions(+), 21 deletions(-) rename src/Common/{TaskStatsInfoGetter.cpp => NetlinkMetricsProvider.cpp} (93%) rename src/Common/{TaskStatsInfoGetter.h => NetlinkMetricsProvider.h} (85%) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 057b1eeda12..ac5b712279e 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -3,7 +3,6 @@ #include "CurrentThread.h" #include #include -#include #include #include #include diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/NetlinkMetricsProvider.cpp similarity index 93% rename from src/Common/TaskStatsInfoGetter.cpp rename to src/Common/NetlinkMetricsProvider.cpp index 867a50c8cce..4c228bcc6fc 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/NetlinkMetricsProvider.cpp @@ -1,4 +1,4 @@ -#include "TaskStatsInfoGetter.h" +#include "NetlinkMetricsProvider.h" #include #include #include @@ -200,7 +200,7 @@ bool checkPermissionsImpl() if (!res) return false; - /// Check that we can successfully initialize TaskStatsInfoGetter. + /// Check that we can successfully initialize NetlinkMetricsProvider. /// It will ask about family id through Netlink. /// On some LXC containers we have capability but we still cannot use Netlink. /// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result. @@ -208,7 +208,7 @@ bool checkPermissionsImpl() try { ::taskstats stats{}; - TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); + NetlinkMetricsProvider().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { @@ -244,14 +244,14 @@ UInt16 getFamilyId(int fd) } -bool TaskStatsInfoGetter::checkPermissions() +bool NetlinkMetricsProvider::checkPermissions() { static bool res = checkPermissionsImpl(); return res; } -TaskStatsInfoGetter::TaskStatsInfoGetter() +NetlinkMetricsProvider::NetlinkMetricsProvider() { netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC); if (netlink_socket_fd < 0) @@ -293,7 +293,7 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() } -void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const +void NetlinkMetricsProvider::getStat(::taskstats & out_stats, pid_t tid) const { NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid)); @@ -318,7 +318,7 @@ void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const } -TaskStatsInfoGetter::~TaskStatsInfoGetter() +NetlinkMetricsProvider::~NetlinkMetricsProvider() { if (netlink_socket_fd >= 0) { @@ -335,15 +335,15 @@ TaskStatsInfoGetter::~TaskStatsInfoGetter() namespace DB { -bool TaskStatsInfoGetter::checkPermissions() +bool NetlinkMetricsProvider::checkPermissions() { return false; } -TaskStatsInfoGetter::TaskStatsInfoGetter() = default; -TaskStatsInfoGetter::~TaskStatsInfoGetter() = default; +NetlinkMetricsProvider::NetlinkMetricsProvider() = default; +NetlinkMetricsProvider::~NetlinkMetricsProvider() = default; -void TaskStatsInfoGetter::getStat(::taskstats &, pid_t) const +void NetlinkMetricsProvider::getStat(::taskstats &, pid_t) const { } diff --git a/src/Common/TaskStatsInfoGetter.h b/src/Common/NetlinkMetricsProvider.h similarity index 85% rename from src/Common/TaskStatsInfoGetter.h rename to src/Common/NetlinkMetricsProvider.h index 66655d7ad0d..8a54f33be80 100644 --- a/src/Common/TaskStatsInfoGetter.h +++ b/src/Common/NetlinkMetricsProvider.h @@ -15,11 +15,11 @@ namespace DB /// /// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101 /// -class TaskStatsInfoGetter : private boost::noncopyable +class NetlinkMetricsProvider : private boost::noncopyable { public: - TaskStatsInfoGetter(); - ~TaskStatsInfoGetter(); + NetlinkMetricsProvider(); + ~NetlinkMetricsProvider(); void getStat(::taskstats & out_stats, pid_t tid) const; diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index a94fd81559a..256f53df011 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -2,7 +2,7 @@ #if defined(OS_LINUX) -#include "TaskStatsInfoGetter.h" +#include "NetlinkMetricsProvider.h" #include "ProcfsMetricsProvider.h" #include "hasLinuxCapability.h" @@ -99,7 +99,7 @@ TasksStatsCounters::MetricsProvider TasksStatsCounters::findBestAvailableProvide static std::optional provider = []() -> MetricsProvider { - if (TaskStatsInfoGetter::checkPermissions()) + if (NetlinkMetricsProvider::checkPermissions()) { return MetricsProvider::Netlink; } @@ -119,7 +119,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p switch (provider) { case MetricsProvider::Netlink: - stats_getter = [metrics_provider = std::make_shared(), tid]() + stats_getter = [metrics_provider = std::make_shared(), tid]() { ::taskstats result{}; metrics_provider->getStat(result, static_cast(tid)); diff --git a/src/Disks/IO/ThreadPoolReader.cpp b/src/Disks/IO/ThreadPoolReader.cpp index effa19bc1af..cd3f2d8dea0 100644 --- a/src/Disks/IO/ThreadPoolReader.cpp +++ b/src/Disks/IO/ThreadPoolReader.cpp @@ -114,7 +114,7 @@ std::future ThreadPoolReader::submit(Request reques /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). Stopwatch watch(CLOCK_MONOTONIC); SCOPE_EXIT({ diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 67bc01279c3..6c0c1681a4c 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -95,7 +95,7 @@ size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_ /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). watch.stop(); ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/SynchronousReader.cpp b/src/IO/SynchronousReader.cpp index 7cef3bd8963..e1c654e48a3 100644 --- a/src/IO/SynchronousReader.cpp +++ b/src/IO/SynchronousReader.cpp @@ -78,7 +78,7 @@ std::future SynchronousReader::submit(Request reque /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). watch.stop(); ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds()); From 19c1f392865c8a8c365a23187943eb4f137674b7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 05:59:25 +0000 Subject: [PATCH 575/871] Refactor code to support storage class only via settings --- src/Backups/registerBackupEngineS3.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 219973d8fc4..34a6c904b8c 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -52,7 +52,6 @@ void registerBackupEngineS3(BackupFactory & factory) const auto & args = params.backup_info.args; String s3_uri, access_key_id, secret_access_key; - String storage_class = params.s3_storage_class; if (!id_arg.empty()) { @@ -77,17 +76,15 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - if ((args.size() != 1) && (args.size() != 3) && (args.size() != 4)) + if ((args.size() != 1) && (args.size() != 3)) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Backup S3 requires 1, 3 or 4 arguments: url, [access_key_id, secret_access_key, [storage_class]]"); + "Backup S3 requires 1 or 3 arguments: url, [access_key_id, secret_access_key]"); s3_uri = args[0].safeGet(); if (args.size() >= 3) { access_key_id = args[1].safeGet(); secret_access_key = args[2].safeGet(); - if (args.size() == 4) - storage_class = args[3].safeGet(); } } @@ -115,7 +112,7 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, storage_class, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.s3_storage_class, params.context); return std::make_unique( backup_name_for_logging, From 51e81b37a4158e31b61c5ae8d993a2dee0ae16e3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 7 Jul 2023 18:45:13 +0200 Subject: [PATCH 576/871] Move condtions with columns from PK to the end of PREWHERE chain --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 31 +++++++++++++++++++ .../MergeTree/MergeTreeWhereOptimizer.h | 8 ++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 1620ba98d58..4ff58c1fc86 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -22,6 +22,33 @@ namespace DB /// This is used to assume that condition is likely to have good selectivity. static constexpr auto threshold = 2; +static NameToIndexMap fillNamesPositions(const Names & names) +{ + NameToIndexMap names_positions; + + for (size_t position = 0; position < names.size(); ++position) + { + const auto & name = names[position]; + names_positions[name] = position; + } + + return names_positions; +} + +/// Find minimal position of any of the column in primary key. +static Int64 findMinPosition(const NameSet & condition_table_columns, const NameToIndexMap & primary_key_positions) +{ + Int64 min_position = std::numeric_limits::max() - 1; + + for (const auto & column : condition_table_columns) + { + auto it = primary_key_positions.find(column); + if (it != primary_key_positions.end()) + min_position = std::min(min_position, static_cast(it->second)); + } + + return min_position; +} MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( std::unordered_map column_sizes_, @@ -35,6 +62,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , supported_columns{supported_columns_} , sorting_key_names{NameSet( metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())} + , primary_key_names_positions(fillNamesPositions(metadata_snapshot->getPrimaryKey().column_names)) , log{log_} , column_sizes{std::move(column_sizes_)} { @@ -234,6 +262,9 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + res.emplace_back(std::move(cond)); } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 18555a72db1..8ab21471aeb 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -72,9 +72,14 @@ private: /// Does the condition presumably have good selectivity? bool good = false; + /// Does the condition contain primary key column? + /// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any + /// column in this condition because this condition have bigger chances to be already satisfied by PK analysis. + Int64 min_position_in_primary_key = std::numeric_limits::max() - 1; + auto tuple() const { - return std::make_tuple(!viable, !good, columns_size, table_columns.size()); + return std::make_tuple(!viable, !good, -min_position_in_primary_key, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -141,6 +146,7 @@ private: const Names queried_columns; const std::optional supported_columns; const NameSet sorting_key_names; + const NameToIndexMap primary_key_names_positions; Poco::Logger * log; std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; From fe7cff5c1cfee89d411ec79e7a3e7603ff831782 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:45:04 +0200 Subject: [PATCH 577/871] Added move_primary_key_columns_to_end_of_prewhere setting --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 11 +++++++++-- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 1 + 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..5a1f4b46223 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -129,6 +129,7 @@ class IColumn; M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \ M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ + M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 4ff58c1fc86..5efb7286685 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -88,6 +88,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons where_optimizer_context.context = context; where_optimizer_context.array_joined_names = determineArrayJoinedNames(select); where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = select.final(); RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); @@ -117,6 +118,7 @@ std::optional MergeTreeWhe where_optimizer_context.context = context; where_optimizer_context.array_joined_names = {}; where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = is_final; RPNBuilderTreeContext tree_context(context); @@ -262,8 +264,13 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); - /// Find min position in PK of any column that is used in this condition. - cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) + { + /// Consider all conditions good with this setting enabled. + cond.good = cond.viable; + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + } res.emplace_back(std::move(cond)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 8ab21471aeb..fb5e84b67c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -96,6 +96,7 @@ private: ContextPtr context; NameSet array_joined_names; bool move_all_conditions_to_prewhere = false; + bool move_primary_key_columns_to_end_of_prewhere = false; bool is_final = false; }; From 695a6b7bb3b6b22a0169e8f786b8cff8ffccc62b Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 06:18:54 +0000 Subject: [PATCH 578/871] Add integration test --- .../test_backup_s3_storage_class/__init__.py | 0 .../test_backup_s3_storage_class/test.py | 51 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 tests/integration/test_backup_s3_storage_class/__init__.py create mode 100644 tests/integration/test_backup_s3_storage_class/test.py diff --git a/tests/integration/test_backup_s3_storage_class/__init__.py b/tests/integration/test_backup_s3_storage_class/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py new file mode 100644 index 00000000000..a4c93d40f0c --- /dev/null +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -0,0 +1,51 @@ +import os +import logging + +import pytest +from helpers.cluster import ClickHouseCluster + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + stay_alive=True, + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_backup_s3_storage_class(started_cluster): + node.query( + """ + CREATE TABLE test_s3_storage_class + ( + `id` UInt64, + `value` String + ) + ENGINE = MergeTree + ORDER BY id; + """, + ) + node.query( + """ + INSERT INTO test_s3_storage_class VALUES (1, 'a'); + """, + ) + result = node.query( + """ + BACKUP TABLE test_s3_storage_class TO S3('http://minio1:9001/root/data', 'minio', 'minio123') + SETTINGS s3_storage_class='STANDARD'; + """ + ) + + assert True From 9b0eb9cdd709418c3782ae2468693b294e81a0cd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:34:44 +0800 Subject: [PATCH 579/871] ignore ast opt when doing projection calc --- src/Storages/ProjectionsDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..086355b6a79 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -298,6 +298,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState} + .ignoreASTOptimizations() .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); From 696818b340d88667a214674f1df483b8c9e827d9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:35:18 +0800 Subject: [PATCH 580/871] Don't check monotonicity when analyze projections --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 4 ++-- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h | 2 +- .../Optimizations/optimizeUseAggregateProjection.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index c9cf46aaeca..1c18465e1e1 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -8,7 +8,7 @@ namespace DB { -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity) { using Parents = std::set; std::unordered_map inner_parents; @@ -182,7 +182,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } } - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + if (!match.node && check_monotonicity && frame.node->function_base->hasInformationAboutMonotonicity()) { size_t num_const_args = 0; const ActionsDAG::Node * monotonic_child = nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h index dd689cba46b..223fc40e33f 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h @@ -39,5 +39,5 @@ struct MatchedTrees using Matches = std::unordered_map; }; -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..cf88de19f03 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -287,7 +287,7 @@ ActionsDAGPtr analyzeAggregateProjection( { auto proj_index = buildDAGIndex(*info.before_aggregation); - MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag); + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag, false /* check_monotonicity */); // for (const auto & [node, match] : matches) // { From 60488e23912ba29ca0e75e2a39b1902517244e6b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:38:42 +0800 Subject: [PATCH 581/871] Add tests --- ...ggregate_projection_with_monotonic_key_expr.reference | 1 + ...1710_aggregate_projection_with_monotonic_key_expr.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql new file mode 100644 index 00000000000..c3109553f63 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS t0; + +CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); + +INSERT INTO t0(c0) VALUES (1); + +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); + +DROP TABLE t0; From d7bb006c231c3960e9c9f7a6f07cd8ba299d6422 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:02:02 +0000 Subject: [PATCH 582/871] Fix monotonic chain for read-in-order as well. --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 7 ++++++- ..._aggregate_projection_with_monotonic_key_expr.reference | 4 ++++ .../01710_aggregate_projection_with_monotonic_key_expr.sql | 7 +++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index 1c18465e1e1..787a106200a 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -75,7 +75,12 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } /// A node from found match may be nullptr. /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); + if (it->second.monotonicity) + /// Ignore a match with monotonicity. + frame.mapped_children.push_back(nullptr); + else + frame.mapped_children.push_back(it->second.node); + } if (frame.mapped_children.size() < frame.node->children.size()) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index d00491fd7e5..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -1 +1,5 @@ 1 +1 +-1 +1 +-1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index c3109553f63..bed43ef6630 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); @@ -6,4 +7,10 @@ INSERT INTO t0(c0) VALUES (1); SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +create table t1 (c0 Int32) engine = MergeTree order by sin(c0); +insert into t1 values (-1), (1); +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; + DROP TABLE t0; +DROP TABLE t1; From ec223372848014b79990bc05318862b8f8e76212 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:54:26 +0800 Subject: [PATCH 583/871] Fix index analysis with indexHint as well --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++++ ...regate_projection_with_monotonic_key_expr.reference | 1 + ...10_aggregate_projection_with_monotonic_key_expr.sql | 10 +++++++++- 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index efab11003f5..b769011e3d4 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -976,7 +976,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (node.name == "indexHint") { if (data.only_consts) + { + /// We need to collect constants inside `indexHint` for index analysis. + if (node.arguments) + { + for (const auto & arg : node.arguments->children) + visit(arg, data); + } return; + } /// Here we create a separate DAG for indexHint condition. /// It will be used only for index analysis. diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 06e9efbe839..1c8b399e790 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,3 +3,4 @@ -1 1 -1 +0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index bed43ef6630..5cd8ec87b4b 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,16 +1,24 @@ DROP TABLE IF EXISTS t0; DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); INSERT INTO t0(c0) VALUES (1); -SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0) SETTINGS optimize_use_implicit_projections = 1; create table t1 (c0 Int32) engine = MergeTree order by sin(c0); insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; +CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + DROP TABLE t0; DROP TABLE t1; +DROP TABLE t2; From 5560603321319243180a7d38b17be29e2f69cf30 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:56:37 +0800 Subject: [PATCH 584/871] optimize_use_implicit_projections=1 by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..b8207b142d3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -577,7 +577,7 @@ class IColumn; M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ - M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \ + M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3e58750e1d2..2886cdd288d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,6 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, From 68e7583dbfaca500757ba0b8e3d3d859b89accfa Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 14:53:06 +0800 Subject: [PATCH 585/871] reorganize tests and add some comments --- .../Optimizations/optimizeUseAggregateProjection.cpp | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- ...aggregate_projection_with_monotonic_key_expr.reference | 1 - ...01710_aggregate_projection_with_monotonic_key_expr.sql | 7 ------- tests/queries/0_stateless/01739_index_hint.reference | 5 +++++ tests/queries/0_stateless/01739_index_hint.sql | 8 ++++++++ 6 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index cf88de19f03..e611bb5b2ef 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -497,6 +497,9 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (block) { MinMaxProjectionCandidate minmax; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..6c1375ecc1d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7027,7 +7027,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg max_added_blocks.get(), query_context); - // minmax_count_projection should not be used when there is no data to process. + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (!query_info.minmax_count_projection_block) return; diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 1c8b399e790..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,4 +3,3 @@ -1 1 -1 -0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index 5cd8ec87b4b..51dafb07b91 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -13,12 +13,5 @@ insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; -CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; - -INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); - -SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; - DROP TABLE t0; DROP TABLE t1; -DROP TABLE t2; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 3a4b380de65..766dff8c7b0 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -33,3 +33,8 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; 100000 drop table XXXX; +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; +0 +drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index e1e66c630e1..77c2760535d 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -33,3 +33,11 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; drop table XXXX; + +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + +drop table XXXX; From 0c2ea94efeeb9a68448d44c1eba08ad3898ef99f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 09:57:32 +0200 Subject: [PATCH 586/871] Update test --- .../0_stateless/02156_storage_merge_prewhere.reference | 2 +- tests/queries/0_stateless/02156_storage_merge_prewhere.sql | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 30f9b1ab175..74ba452d783 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,6 +1,6 @@ SELECT count() FROM t_02156_merge1 -PREWHERE (k = 3) AND notEmpty(v) +PREWHERE notEmpty(v) AND (k = 3) 2 SELECT count() FROM t_02156_merge2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index b75d3fa22e5..83d88a68d9b 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -1,4 +1,5 @@ SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; DROP TABLE IF EXISTS t_02156_mt1; DROP TABLE IF EXISTS t_02156_mt2; @@ -8,8 +9,8 @@ DROP TABLE IF EXISTS t_02156_merge1; DROP TABLE IF EXISTS t_02156_merge2; DROP TABLE IF EXISTS t_02156_merge3; -CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; -CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; +CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; +CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; CREATE TABLE t_02156_log (k UInt32, v String) ENGINE = Log; CREATE TABLE t_02156_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_mt1); From d2dba496bf0c703178758b1c534c0914044d2094 Mon Sep 17 00:00:00 2001 From: StianBerger <111980234+StianBerger@users.noreply.github.com> Date: Fri, 21 Jul 2023 10:26:01 +0200 Subject: [PATCH 587/871] Update date-time-functions.md formatDateTime %r for 12-hour time, mentioned %H in equivalent, which is 24H. Replaced with %h. --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c6b978506a1..87d84425029 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1449,7 +1449,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | | %Q | Quarter (1-4) | 1 | -| %r | 12-hour HH:MM AM/PM time, equivalent to %H:%i %p | 10:30 PM | +| %r | 12-hour HH:MM AM/PM time, equivalent to %h:%i %p | 10:30 PM | | %R | 24-hour HH:MM time, equivalent to %H:%i | 22:33 | | %s | second (00-59) | 44 | | %S | second (00-59) | 44 | From 53d77e6b1397e3621a81fc88da76aa9bac72ad75 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 21:28:17 +0800 Subject: [PATCH 588/871] Add back missing projection QueryAccessInfo. --- src/Interpreters/Context.cpp | 13 ++-- src/Interpreters/Context.h | 1 + .../optimizeUseAggregateProjection.cpp | 18 +++++- .../optimizeUseNormalProjection.cpp | 9 ++- .../QueryPlan/ReadFromMergeTree.cpp | 4 ++ .../QueryPlan/ReadFromPreparedSource.cpp | 8 ++- .../QueryPlan/ReadFromPreparedSource.h | 3 +- ...0_query_log_with_projection_info.reference | 3 + .../01710_query_log_with_projection_info.sql | 64 +++++++++++++++++++ 9 files changed, 114 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/01710_query_log_with_projection_info.reference create mode 100644 tests/queries/0_stateless/01710_query_log_with_projection_info.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e4d1e8d1e2..434fc1adb40 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1461,15 +1461,20 @@ void Context::addQueryAccessInfo( void Context::addQueryAccessInfo(const Names & partition_names) { if (isGlobalContext()) - { throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); - } std::lock_guard lock(query_access_info.mutex); for (const auto & partition_name : partition_names) - { query_access_info.partitions.emplace(partition_name); - } +} + +void Context::addQueryAccessInfo(const String & qualified_projection_name) +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); + + std::lock_guard lock(query_access_info.mutex); + query_access_info.projections.emplace(qualified_projection_name); } void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3a8d41bf130..3ce899bfb77 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -657,6 +657,7 @@ public: const String & projection_name = {}, const String & view_name = {}); void addQueryAccessInfo(const Names & partition_names); + void addQueryAccessInfo(const String & qualified_projection_name); /// Supported factories for records in query_log diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..8c85435138c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -625,7 +625,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & // candidates.minmax_projection->block.dumpStructure()); Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(candidates.minmax_projection->candidate.projection->name))); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) @@ -658,7 +665,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & { auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); Pipe pipe(std::make_shared(std::move(header))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(best_candidate->projection->name))); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index dd7a5d449bc..71db561e1c9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -183,7 +183,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!projection_reading) { Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(best_candidate->projection->name))); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 13de5d1d140..82f47cc61d5 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1761,6 +1761,10 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); } context->getQueryContext()->addQueryAccessInfo(partition_names); + + if (storage_snapshot->projection) + context->getQueryContext()->addQueryAccessInfo( + fmt::format("{}.{}", data.getStorageID().getFullTableName(), backQuoteIfNeed(storage_snapshot->projection->name))); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 7446203ec35..d50eec47ca8 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,17 +1,23 @@ +#include #include #include namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, const String & qualified_projection_name_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) + , context(context_) + , qualified_projection_name(qualified_projection_name_) { } void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + if (context && context->hasQueryContext() && !qualified_projection_name.empty()) + context->getQueryContext()->addQueryAccessInfo(qualified_projection_name); + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 05e3ebd5102..5e64dcb7a4f 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -9,7 +9,7 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource(Pipe pipe_); + explicit ReadFromPreparedSource(Pipe pipe_, ContextPtr context_ = nullptr, const String & qualified_projection_name_ = ""); String getName() const override { return "ReadFromPreparedSource"; } @@ -18,6 +18,7 @@ public: protected: Pipe pipe; ContextPtr context; + String qualified_projection_name; }; class ReadFromStorageStep : public ReadFromPreparedSource diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.reference b/tests/queries/0_stateless/01710_query_log_with_projection_info.reference new file mode 100644 index 00000000000..9c2e9df6662 --- /dev/null +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.reference @@ -0,0 +1,3 @@ +t.t_normal +t.t_agg +t._minmax_count_projection diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql new file mode 100644 index 00000000000..25e7e8fed60 --- /dev/null +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql @@ -0,0 +1,64 @@ +set log_queries=1; +set log_queries_min_type='QUERY_FINISH'; +set optimize_use_implicit_projections=1; + +DROP TABLE IF EXISTS t; + +CREATE TABLE t +( + `id` UInt64, + `id2` UInt64, + `id3` UInt64, + PROJECTION t_normal + ( + SELECT + id, + id2, + id3 + ORDER BY + id2, + id, + id3 + ), + PROJECTION t_agg + ( + SELECT + sum(id3) + GROUP BY id2 + ) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS index_granularity = 8; + +insert into t SELECT number, -number, number FROM numbers(10000); + +SELECT * FROM t WHERE id2 = 3 FORMAT Null; +SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null; +SELECT min(id) FROM t FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT * FROM t WHERE id2 = 3 FORMAT Null;'; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null;'; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT min(id) FROM t FORMAT Null;'; From 8187118232371630fb10ee4062b8a52285003fa0 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 20 Jul 2023 11:12:22 +0800 Subject: [PATCH 589/871] Better code --- src/Interpreters/Context.cpp | 8 ++++++-- src/Interpreters/Context.h | 9 ++++++++- .../optimizeUseAggregateProjection.cpp | 18 ++++++++---------- .../optimizeUseNormalProjection.cpp | 9 ++++----- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- .../QueryPlan/ReadFromPreparedSource.cpp | 9 ++++----- .../QueryPlan/ReadFromPreparedSource.h | 7 +++++-- 7 files changed, 36 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 434fc1adb40..cc1277e08b9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1468,13 +1468,17 @@ void Context::addQueryAccessInfo(const Names & partition_names) query_access_info.partitions.emplace(partition_name); } -void Context::addQueryAccessInfo(const String & qualified_projection_name) +void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name) { + if (!qualified_projection_name) + return; + if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); std::lock_guard lock(query_access_info.mutex); - query_access_info.projections.emplace(qualified_projection_name); + query_access_info.projections.emplace(fmt::format( + "{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name))); } void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3ce899bfb77..fa210f04451 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -657,7 +657,14 @@ public: const String & projection_name = {}, const String & view_name = {}); void addQueryAccessInfo(const Names & partition_names); - void addQueryAccessInfo(const String & qualified_projection_name); + + struct QualifiedProjectionName + { + StorageID storage_id = StorageID::createEmpty(); + String projection_name; + explicit operator bool() const { return !projection_name.empty(); } + }; + void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name); /// Supported factories for records in query_log diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 8c85435138c..fa6a7f5b8ea 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -628,11 +628,10 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(candidates.minmax_projection->candidate.projection->name))); + query_info.is_internal ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = candidates.minmax_projection->candidate.projection->name}); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) @@ -668,11 +667,10 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(best_candidate->projection->name))); + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 71db561e1c9..93d1be20e81 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -186,11 +186,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(best_candidate->projection->name))); + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 82f47cc61d5..2d2412f7e36 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1764,7 +1764,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (storage_snapshot->projection) context->getQueryContext()->addQueryAccessInfo( - fmt::format("{}.{}", data.getStorageID().getFullTableName(), backQuoteIfNeed(storage_snapshot->projection->name))); + Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index d50eec47ca8..a24c4dbe4d0 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,21 +1,20 @@ -#include #include #include namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, const String & qualified_projection_name_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) - , context(context_) - , qualified_projection_name(qualified_projection_name_) + , context(std::move(context_)) + , qualified_projection_name(std::move(qualified_projection_name_)) { } void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - if (context && context->hasQueryContext() && !qualified_projection_name.empty()) + if (context && context->hasQueryContext()) context->getQueryContext()->addQueryAccessInfo(qualified_projection_name); for (const auto & processor : pipe.getProcessors()) diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 5e64dcb7a4f..2606f501009 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -1,4 +1,6 @@ #pragma once + +#include #include #include @@ -9,7 +11,8 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource(Pipe pipe_, ContextPtr context_ = nullptr, const String & qualified_projection_name_ = ""); + explicit ReadFromPreparedSource( + Pipe pipe_, ContextPtr context_ = nullptr, Context::QualifiedProjectionName qualified_projection_name_ = {}); String getName() const override { return "ReadFromPreparedSource"; } @@ -18,7 +21,7 @@ public: protected: Pipe pipe; ContextPtr context; - String qualified_projection_name; + Context::QualifiedProjectionName qualified_projection_name; }; class ReadFromStorageStep : public ReadFromPreparedSource From 2cad81731be0443b50e66e43fb68b2b064d67a77 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 16:46:56 +0800 Subject: [PATCH 590/871] Try to fix style issues --- .../optimizeUseAggregateProjection.cpp | 19 ++++++++++++------- .../optimizeUseNormalProjection.cpp | 7 +++++-- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index fa6a7f5b8ea..53f47bcdf95 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -628,11 +628,13 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = candidates.minmax_projection->candidate.projection->name}); - + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = candidates.minmax_projection->candidate.projection->name, + }); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) reading->resetParts(std::move(candidates.minmax_projection->normal_parts)); @@ -669,8 +671,11 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & context, query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 93d1be20e81..f6ace6f8025 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -188,8 +188,11 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) context, query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; From 5fa45bdbeaef99ba6a7db894d89dc749b7ac3f97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:12:34 +0200 Subject: [PATCH 591/871] Setting the metric FilesystemCacheSizeLimit in LRUFileCachePriority.h --- src/Interpreters/Cache/LRUFileCachePriority.h | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index e0d7d45062a..662a76968bc 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -5,6 +5,12 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSizeLimit; +} + + namespace DB { @@ -18,7 +24,9 @@ private: using LRUQueueIterator = typename LRUQueue::iterator; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {} + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) { + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); + } size_t getSize(const CacheGuard::Lock &) const override { return current_size; } From 930d45303c5b96b7553d611e82e0c94215ef5705 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:13:38 +0200 Subject: [PATCH 592/871] removing the metric set from LRUFileCachePriority.cpp --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 33e567b7a76..18862e154da 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -7,7 +7,6 @@ namespace CurrentMetrics { extern const Metric FilesystemCacheSize; - extern const Metric FilesystemCacheSizeLimit; extern const Metric FilesystemCacheElements; } @@ -102,7 +101,6 @@ void LRUFileCachePriority::updateSize(int64_t size) { current_size += size; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); - CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, getSizeLimit()); } void LRUFileCachePriority::updateElementsCount(int64_t num) From 3412dd225919f3850dfb4c0f8647e74e6630e31f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:14:30 +0200 Subject: [PATCH 593/871] removed unnecessary whitespace --- src/Interpreters/Cache/LRUFileCachePriority.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 662a76968bc..9396070b792 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -10,7 +10,6 @@ namespace CurrentMetrics extern const Metric FilesystemCacheSizeLimit; } - namespace DB { From 714a3a8d121326e2d908648bfc0e76e09f6e0815 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 12:23:22 +0200 Subject: [PATCH 594/871] 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 06f5330f6d9..841b646a126 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 e638a9ecd3cebe83c0c3997b19e0e73d1fb14639 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 Jul 2023 12:24:36 +0200 Subject: [PATCH 595/871] Fix style check --- src/Interpreters/Cache/LRUFileCachePriority.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 9396070b792..e041e59a91a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -23,7 +23,8 @@ private: using LRUQueueIterator = typename LRUQueue::iterator; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) { + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) + { CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); } From 3815a6aa62414fcd4e51090450a08bc368fd8c86 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Jul 2023 11:14:39 +0000 Subject: [PATCH 596/871] Make test_keeper_force_recovery more stable --- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/enable_keeper4.xml | 2 ++ .../configs/enable_keeper5.xml | 2 ++ .../configs/enable_keeper6.xml | 2 ++ .../configs/enable_keeper7.xml | 2 ++ .../configs/enable_keeper8.xml | 2 ++ .../configs/recovered_keeper1.xml | 2 ++ .../configs/use_keeper.xml | 36 ------------------- .../test_keeper_force_recovery/test.py | 5 +-- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper1_solo.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/use_keeper.xml | 16 --------- .../test.py | 5 +-- 17 files changed, 28 insertions(+), 60 deletions(-) delete mode 100644 tests/integration/test_keeper_force_recovery/configs/use_keeper.xml delete mode 100644 tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml index b7f9d1b058e..62109ee9092 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml index b773d59f259..2696c573180 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml index d4c2befc10f..fc0c0fd0300 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml index c039e709c9e..06f1c1d7195 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml @@ -1,5 +1,6 @@ + false 9181 4 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml index fb43b6524c8..5d3767ae969 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml @@ -1,5 +1,6 @@ + false 9181 5 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml index 430e662bf36..4d30822741a 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml @@ -1,5 +1,6 @@ + false 9181 6 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml index aa10774ad7d..b59141042ea 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml @@ -1,5 +1,6 @@ + false 9181 7 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml index 4f1c21a1084..711d70cb1ac 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml @@ -1,5 +1,6 @@ + false 9181 8 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml index eaf0f01afc9..abd4ef85bf7 100644 --- a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml deleted file mode 100644 index f41e8c6e49c..00000000000 --- a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - node4 - 9181 - - - node5 - 9181 - - - node6 - 9181 - - - node7 - 9181 - - - node8 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index f7c3787b4d8..f630e5a422b 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -22,10 +22,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml index 441c1bc185d..94e59128bd3 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml index f0cb887b062..6367b4b4c29 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml @@ -1,5 +1,6 @@ + false 1 9181 1 @@ -11,6 +12,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml index e2e2c1fd7db..548d12c2e0a 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml index e2ac0400d88..65f9675cbd6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml deleted file mode 100644 index 384e984f210..00000000000 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery_single_node/test.py b/tests/integration/test_keeper_force_recovery_single_node/test.py index 1c0d5e9a306..132c5488df6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/test.py +++ b/tests/integration/test_keeper_force_recovery_single_node/test.py @@ -20,10 +20,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) From 8970a531453cbfd9916681bf658ba605db0129de Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:38:46 +0000 Subject: [PATCH 597/871] Fix `countSubstrings()` hang with empty needle and a column haystack --- src/Functions/CountSubstringsImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index de00e9397d6..1501e50afcf 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -49,6 +49,9 @@ struct CountSubstringsImpl /// FIXME: suboptimal memset(&res[0], 0, res.size() * sizeof(res[0])); + if (needle.empty()) + return; // Return all zeros + /// Current index in the array of strings. size_t i = 0; From 66c8066cd59a685ded94af69d2cc37469cb29d77 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:45:44 +0000 Subject: [PATCH 598/871] add test --- tests/queries/0_stateless/01590_countSubstrings.reference | 2 ++ tests/queries/0_stateless/01590_countSubstrings.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 056cb4c53b6..95031cd3856 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -7,6 +7,8 @@ empty 0 0 0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index dd2122ed6ff..6d2d87b1260 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -12,6 +12,8 @@ select 'empty'; select countSubstrings('', '.'); select countSubstrings('', ''); select countSubstrings('.', ''); +select countSubstrings(toString(number), '') from numbers(1); +select countSubstrings('', toString(number)) from numbers(1); select 'char'; select countSubstrings('foobar.com', '.'); From 0f969923229375d72faac15257fc70bd7ece9095 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Jul 2023 08:07:50 +0200 Subject: [PATCH 599/871] Fix possible EADDRINUSE ("Address already in use") in integration tests Here is one example [1]: minio1_1 | WARNING: Console endpoint is listening on a dynamic port (32911), please use --console-address ":PORT" to choose a static port. minio1_1 | ERROR Unable to initialize console server: Specified port is already in use minio1_1 | > Please ensure no other program uses the same address/port [1]: https://s3.amazonaws.com/clickhouse-test-reports/52103/7d510eac7c5f0dfb3361e269be30972e6022fada/integration_tests__tsan__[1_6].html And here is second [2]: java.net.BindException: Problem binding to [0.0.0.0:50020] java.net.BindException: Address already in use; For more details see: http://wiki.apache.org/hadoop/BindException [2]: https://s3.amazonaws.com/clickhouse-test-reports/51493/63e88b725d3d255a6534adce4d434ce5f95d2874/integration_tests__asan__[1_6].html v2: increase the limit from 5K to 10K Signed-off-by: Azat Khuzhin --- tests/integration/conftest.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5933883f7b0..968571bfdde 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -12,6 +12,22 @@ from helpers.network import _NetworkManager logging.raiseExceptions = False +@pytest.fixture(autouse=True, scope="session") +def tune_local_port_range(): + # Lots of services uses non privileged ports: + # - hdfs -- 50020/50070/... + # - minio + # - mysql + # - psql + # + # So instead of tuning all these thirdparty services, let's simply + # prohibit using such ports for outgoing connections, this should fix + # possible "Address already in use" errors. + # + # NOTE: 5K is not enough, and sometimes leads to EADDRNOTAVAIL error. + run_and_check(["sysctl net.ipv4.ip_local_port_range='55000 65535'"], shell=True) + + @pytest.fixture(autouse=True, scope="session") def cleanup_environment(): try: From 1fe8076b9422e09b78887b44a724cf1cae5d7fd8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 20 Jul 2023 12:38:57 +0200 Subject: [PATCH 600/871] Fix capabilities installed via systemd service (fixes netlink/IO priorities) CapabilityBoundingSet that contained in systemd unit before is about allowing to set some capabilities, not about granting them. To grant them you need to use AmbientCapabilities. And if you do not use 'clickhouse install' then: - IO priorities was unavailable (since they requires CAP_SYS_NICE) - For taskstats the procfs was used instead of netlink Not a big deal, but still. Here how it had been tested: $ systemd-run -p CapabilityBoundingSet=CAP_NET_ADMIN --shell root:/etc (master)# capsh --print Current: cap_net_admin=ep Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: = Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p AmbientCapabilities=CAP_NET_ADMIN -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: cap_net_admin=eip Bounding set =cap_net_admin Ambient set =cap_net_admin Note, if you are running it under root (without changing user) you don't need to specify AmbientCapabilities additionally, because root has all capabilities by default and they had been inherited. Signed-off-by: Azat Khuzhin --- packages/clickhouse-server.service | 1 + 1 file changed, 1 insertion(+) diff --git a/packages/clickhouse-server.service b/packages/clickhouse-server.service index 7742d8b278a..42dc5bd380d 100644 --- a/packages/clickhouse-server.service +++ b/packages/clickhouse-server.service @@ -29,6 +29,7 @@ EnvironmentFile=-/etc/default/clickhouse LimitCORE=infinity LimitNOFILE=500000 CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE +AmbientCapabilities=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE [Install] # ClickHouse should not start from the rescue shell (rescue.target). From b45c2c939b974ea3306f9c2192f362d71a69c0e2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 21 Jul 2023 15:17:07 +0300 Subject: [PATCH 601/871] disable expression templates for time intervals (#52335) --- .../Impl/ConstantExpressionTemplate.cpp | 33 +++++++++++++++++++ .../Formats/Impl/ConstantExpressionTemplate.h | 2 ++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 5 +++ ...2830_insert_values_time_interval.reference | 4 +++ .../02830_insert_values_time_interval.sql | 25 ++++++++++++++ 5 files changed, 69 insertions(+) create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.reference create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.sql diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..06efe0a20aa 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -177,6 +177,14 @@ private: if (function.name == "lambda") return; + /// Parsing of INTERVALs is quite hacky. Expressions are rewritten during parsing like this: + /// "now() + interval 1 day" -> "now() + toIntervalDay(1)" + /// "select now() + INTERVAL '1 day 1 hour 1 minute'" -> "now() + (toIntervalDay(1), toIntervalHour(1), toIntervalMinute(1))" + /// so the AST is completely different from the original expression . + /// Avoid extracting these literals and simply compare tokens. It makes the template less flexible but much simpler. + if (function.name.starts_with("toInterval")) + return; + FunctionOverloadResolverPtr builder = FunctionFactory::instance().get(function.name, context); /// Do not replace literals which must be constant ColumnNumbers dont_visit_children = builder->getArgumentsThatAreAlwaysConstant(); @@ -350,6 +358,31 @@ ConstantExpressionTemplate::TemplateStructure::TemplateStructure(LiteralsInfo & } +String ConstantExpressionTemplate::TemplateStructure::dumpTemplate() const +{ + WriteBufferFromOwnString res; + + size_t cur_column = 0; + size_t cur_token = 0; + size_t num_columns = literals.columns(); + while (cur_column < num_columns) + { + size_t skip_tokens_until = token_after_literal_idx[cur_column]; + while (cur_token < skip_tokens_until) + res << quote << tokens[cur_token++] << ", "; + + const DataTypePtr & type = literals.getByPosition(cur_column).type; + res << type->getName() << ", "; + ++cur_column; + } + + while (cur_token < tokens.size()) + res << quote << tokens[cur_token++] << ", "; + + res << "eof"; + return res.str(); +} + size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index fbb3cbcd22a..71d0d0f7134 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -31,6 +31,8 @@ class ConstantExpressionTemplate : boost::noncopyable static size_t getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, bool null_as_default, const String & salt); + String dumpTemplate() const; + String result_column_name; std::vector tokens; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index d61e723fd75..3a65a6fe4ea 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -474,6 +475,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx context, &found_in_cache, delimiter); + + LOG_TEST(&Poco::Logger::get("ValuesBlockInputFormat"), "Will use an expression template to parse column {}: {}", + column_idx, structure->dumpTemplate()); + templates[column_idx].emplace(structure); if (found_in_cache) ++attempts_to_deduce_template_cached[column_idx]; diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.reference b/tests/queries/0_stateless/02830_insert_values_time_interval.reference new file mode 100644 index 00000000000..b5b57fbfbfe --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.reference @@ -0,0 +1,4 @@ +1 2023-07-21 22:54:02 +2 2023-07-21 21:53:01 +3 2023-07-21 21:53:01 +4 2023-07-20 21:54:02 diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.sql b/tests/queries/0_stateless/02830_insert_values_time_interval.sql new file mode 100644 index 00000000000..f5d5d8a4c04 --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.sql @@ -0,0 +1,25 @@ + +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + c1 DateTime DEFAULT now() NOT NULL, + c2 DateTime DEFAULT now() NOT NULL, + c3 DateTime DEFAULT now() NOT NULL, + PRIMARY KEY(c1, c2, c3) +) ENGINE = MergeTree() +ORDER BY (c1, c2, c3); + +INSERT INTO t1 (c1,c2,c3) VALUES(now() + INTERVAL '1 day 1 hour 1 minute 1 second', now(), now()); + +DROP TABLE t1; + +CREATE TABLE t1 (n int, dt DateTime) ENGINE=Memory; + +SET input_format_values_interpret_expressions=0; +INSERT INTO t1 VALUES (1, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day 1 hour 1 minute 1 second'), (2, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day'); +INSERT INTO t1 VALUES (3, toDateTime('2023-07-20 21:53:01') + INTERVAL 1 DAY), (4, toDateTime('2023-07-20 21:53:01') + (toIntervalMinute(1), toIntervalSecond(1))); + +SELECT * FROM t1 ORDER BY n; + +DROP TABLE t1; From 3798bd6f509b7fc0591201c66c2e0d1b254835dd Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 12:52:07 +0000 Subject: [PATCH 602/871] 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 2d8c5a5f61f..503802016cb 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 8ec8388a9ef063beb02b430ae4b89dfe5bab9ddd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 Jul 2023 14:53:02 +0200 Subject: [PATCH 603/871] Update gtest_lru_file_cache.cpp --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index b9d12c8ed42..12e7d9372f7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -470,6 +470,7 @@ TEST_F(FileCacheTest, get) auto & file_segment2 = get(holder2, 2); ASSERT_TRUE(file_segment2.getOrSetDownloader() != FileSegment::getCallerId()); + ASSERT_EQ(file_segment2.state(), State::DOWNLOADING); { std::lock_guard lock(mutex); @@ -478,8 +479,7 @@ TEST_F(FileCacheTest, get) cv.notify_one(); file_segment2.wait(file_segment2.range().right); - file_segment2.complete(); - ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED); + ASSERT_EQ(file_segment2.getDownloadedSize(false), file_segment2.range().size()); }); { @@ -488,7 +488,8 @@ TEST_F(FileCacheTest, get) } download(file_segment); - ASSERT_TRUE(file_segment.state() == State::DOWNLOADED); + ASSERT_EQ(file_segment.state(), State::DOWNLOADED); + file_segment.completePartAndResetDownloader(); other_1.join(); From 4695ec6802c80d25e93a7b523821840c10a3b200 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 14:56:29 +0200 Subject: [PATCH 604/871] Add an ability to specify allocations size for sampling memory profiler --- programs/server/Server.cpp | 21 ++++++++++----- src/Common/MemoryTracker.cpp | 11 ++++++-- src/Common/MemoryTracker.h | 18 +++++++++++++ src/Core/ServerSettings.h | 8 ++++-- src/Core/Settings.h | 4 ++- src/Interpreters/ProcessList.cpp | 3 +++ src/Interpreters/ThreadStatusExt.cpp | 2 ++ .../__init__.py | 1 + .../configs/max_untracked_memory.xml | 7 +++++ .../configs/memory_profiler.xml | 5 ++++ .../test.py | 27 +++++++++++++++++++ ...r_sample_min_max_allocation_size.reference | 1 + ...profiler_sample_min_max_allocation_size.sh | 18 +++++++++++++ 13 files changed, 115 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py create mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference create mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 948824242fb..71bf8cc9e89 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1643,17 +1643,26 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); - if (total_memory_profiler_step) + if (server_settings.total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(total_memory_profiler_step); + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); } - double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); - if (total_memory_tracker_sample_probability > 0.0) + if (server_settings.total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 81cac2617c5..52cd9cc8073 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,6 +534,13 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } +bool MemoryTracker::isSizeOkForSampling(UInt64 size) const +{ + //LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "CHECKING SIZE {} IN BORDERS [{}; {}]", size, min_allocation_size_bytes, max_allocation_size_bytes); + /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation + return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); +} + bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4e29d40c953..768dc8a7404 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,6 +67,12 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; + /// Randomly sample allocations only larger or equal to this size + UInt64 min_allocation_size_bytes = 0; + + /// Randomly sample allocations only smaller or equal to this size + UInt64 max_allocation_size_bytes = 0; + /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -88,6 +94,8 @@ private: void setOrRaiseProfilerLimit(Int64 value); + bool isSizeOkForSampling(UInt64 size) const; + /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -165,6 +173,16 @@ public: sample_probability = value; } + void setSampleMinAllocationSize(UInt64 value) + { + min_allocation_size_bytes = value; + } + + void setSampleMaxAllocationSize(UInt64 value) + { + max_allocation_size_bytes = value; + } + void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1a9f226041b..f7a6c9e950e 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -81,8 +81,12 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) - + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ + \ + M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ + M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..bcfc179be5e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -426,7 +426,9 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1503e396298..c299572ef41 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); + thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 5acfe500b1d..49d9d3ccdf6 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml new file mode 100644 index 00000000000..56fc5ed34ca --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml new file mode 100644 index 00000000000..5b3e17d145f --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml @@ -0,0 +1,5 @@ + + 1 + 4096 + 8192 + diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py new file mode 100644 index 00000000000..b768a442591 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -0,0 +1,27 @@ +from helpers.cluster import ClickHouseCluster +import pytest + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/memory_profiler.xml"], + user_configs=["configs/max_untracked_memory.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_trace_boundaries_work(started_cluster): + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") + node.query("SYSTEM FLUSH LOGS") + + assert node.query("SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'") == "1\n" + assert node.query("SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)") == "0\n" diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh new file mode 100755 index 00000000000..b1fbea26da7 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings +# requires TraceCollector, does not available under sanitizers and aarch64 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" +${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + +# at least something allocated +${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" + +# show wrong allocations +${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From 0aed62ec73b8de4614506f5b72a086d8d10db4aa Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:03:25 +0000 Subject: [PATCH 605/871] 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 fb870ababa3..3f4e35a78a4 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 07bf69d17c8..5f7769f7403 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 e0fbd4b2948..62610964502 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 8d4c840e2da0401787bafa2239907ff59160a003 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:15:35 +0200 Subject: [PATCH 606/871] Some more cases --- tests/queries/0_stateless/01590_countSubstrings.reference | 3 +++ tests/queries/0_stateless/01590_countSubstrings.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 95031cd3856..367b910e569 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -9,6 +9,9 @@ empty 0 0 0 +0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index 6d2d87b1260..b38cbb7d188 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -14,6 +14,9 @@ select countSubstrings('', ''); select countSubstrings('.', ''); select countSubstrings(toString(number), '') from numbers(1); select countSubstrings('', toString(number)) from numbers(1); +select countSubstrings('aaa', materialize('')); +select countSubstrings(materialize('aaa'), ''); +select countSubstrings(materialize('aaa'), materialize('')); select 'char'; select countSubstrings('foobar.com', '.'); From b5cf64466887e115656aab065848fb52784964ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 21 Jul 2023 15:23:04 +0200 Subject: [PATCH 607/871] Check projection metadata the same way we check ordinary metadata. (#52361) * Check projection metadata the same way we check ordinary metadata. * Allow aggregate projection to have empty PK --------- Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../02540_duplicate_primary_key.sql | 18 +----------------- .../02540_duplicate_primary_key2.reference | 1 - .../02540_duplicate_primary_key2.sql | 10 +--------- .../02816_check_projection_metadata.reference | 0 .../02816_check_projection_metadata.sql | 3 +++ 7 files changed, 13 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.reference create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a820bacf9a3..34be8156e71 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -465,9 +465,10 @@ void MergeTreeData::checkProperties( const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, + bool allow_empty_sorting_key, ContextPtr local_context) const { - if (!new_metadata.sorting_key.definition_ast) + if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty"); KeyDescription new_sorting_key = new_metadata.sorting_key; @@ -580,6 +581,9 @@ void MergeTreeData::checkProperties( if (projections_names.find(projection.name) != projections_names.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection with name {} already exists", backQuote(projection.name)); + /// We cannot alter a projection so far. So here we do not try to find a projection in old metadata. + bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate; + checkProperties(*projection.metadata, *projection.metadata, attach, is_aggregate, local_context); projections_names.insert(projection.name); } } @@ -593,7 +597,7 @@ void MergeTreeData::setProperties( bool attach, ContextPtr local_context) { - checkProperties(new_metadata, old_metadata, attach, local_context); + checkProperties(new_metadata, old_metadata, attach, false, local_context); setInMemoryMetadata(new_metadata); } @@ -3286,7 +3290,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } - checkProperties(new_metadata, old_metadata, false, local_context); + checkProperties(new_metadata, old_metadata, false, false, local_context); checkTTLExpressions(new_metadata, old_metadata); if (!columns_to_check_conversion.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8b5b50b1841..28611d09386 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1229,7 +1229,7 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr) const; + void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, bool allow_empty_sorting_key, ContextPtr local_context) const; void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr); diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key.sql b/tests/queries/0_stateless/02540_duplicate_primary_key.sql index a084d76964b..6905c9d5133 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key.sql @@ -86,20 +86,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(coverage) -ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); - -insert into test select * from generateRandom() limit 10; - -with dissonance as ( - Select cast(toStartOfInterval(coverage, INTERVAL 1 day) as Date) as flour, count() as regulation - from test - group by flour having flour >= toDate(now())-100 - ), -cheetah as ( - Select flour, regulation from dissonance - union distinct - Select toDate(now())-1, ifnull((select regulation from dissonance where flour = toDate(now())-1),0) as regulation -) -Select flour, regulation from cheetah order by flour with fill step 1 limit 100 format Null; - -drop table test; +ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference index 08839f6bb29..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference @@ -1 +0,0 @@ -200 diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql index d0f02a894f2..53800c95550 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql @@ -88,12 +88,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(timestamp) -ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); - -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'Airtel' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'BSNL' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'xxx' as xxxx14 FROM generateRandom() LIMIT 100; - -select sum(1) from test where toStartOfInterval(timestamp, INTERVAL 1 day) >= TIMESTAMP '2022-02-01 01:00:00' and xxxx14 in ('Airtel', 'BSNL') and xxxx1 = 1 GROUP BY xxxx16; - -drop table test; +ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); -- { serverError BAD_ARGUMENTS} diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.reference b/tests/queries/0_stateless/02816_check_projection_metadata.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.sql b/tests/queries/0_stateless/02816_check_projection_metadata.sql new file mode 100644 index 00000000000..e7da043ad41 --- /dev/null +++ b/tests/queries/0_stateless/02816_check_projection_metadata.sql @@ -0,0 +1,3 @@ +create table kek (uuid FixedString(16), id int, ns String, dt DateTime64(6), projection null_pk (select * order by ns, 1, 4)) engine=MergeTree order by (id, dt, uuid); -- {serverError ILLEGAL_COLUMN } +-- this query could segfault or throw LOGICAL_ERROR previously, when we did not check projection PK +-- insert into kek select * from generageRandom(10000); From cb53d762eae54eef9411e2cf7548927f83fe187b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:23:24 +0200 Subject: [PATCH 608/871] Fix one more case --- src/Functions/CountSubstringsImpl.h | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 1501e50afcf..8ba9ee99de8 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -226,16 +226,19 @@ struct CountSubstringsImpl const char * needle_beg = reinterpret_cast(&needle_data[prev_needle_offset]); size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); - - const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); - const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); - - const UInt8 * pos; - while ((pos = searcher.search(beg, end)) < end) + if (needle_size > 0) { - ++res[i]; - beg = pos + needle_size; + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); + + const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); + const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); + + const UInt8 * pos; + while ((pos = searcher.search(beg, end)) < end) + { + ++res[i]; + beg = pos + needle_size; + } } } From 10ec06917506c1a61caadf4c680bd0148520426f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:29:40 +0000 Subject: [PATCH 609/871] 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 6529e94a41d..73fc5c58b2f 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 62610964502..da32a8f0ac8 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 610/871] 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 da32a8f0ac8..b148f9a051a 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 abd8bfed2b6e6c20b46ffbeb82699c8530523ffe Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 15:44:49 +0200 Subject: [PATCH 611/871] Remove comment --- src/Common/MemoryTracker.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 52cd9cc8073..52cae0768dc 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -536,7 +536,6 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) bool MemoryTracker::isSizeOkForSampling(UInt64 size) const { - //LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "CHECKING SIZE {} IN BORDERS [{}; {}]", size, min_allocation_size_bytes, max_allocation_size_bytes); /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); } From c080e9b450faeaced13c149212456ab006648c3a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 21:48:49 +0800 Subject: [PATCH 612/871] Fix normal projection with merge table --- .../Optimizations/optimizeUseNormalProjection.cpp | 8 ++++++-- ..._projection_query_plan_optimization_misc.reference | 1 + .../01710_projection_query_plan_optimization_misc.sql | 11 +++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference create mode 100644 tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index dd7a5d449bc..2a03a082d89 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -92,6 +92,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) break; } + /// Dangling query plan node. This might be generated by StorageMerge. + if (iter->node->step.get() == reading) + return false; + const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -105,8 +109,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) QueryDAG query; { - auto & clild = iter->node->children[iter->next_child - 1]; - if (!query.build(*clild)) + auto & child = iter->node->children[iter->next_child - 1]; + if (!query.build(*child)) return false; if (query.dag) diff --git a/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference new file mode 100644 index 00000000000..9874d6464ab --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference @@ -0,0 +1 @@ +1 2 diff --git a/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql new file mode 100644 index 00000000000..cb565313380 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql @@ -0,0 +1,11 @@ +drop table if exists t; + +create table t (x Int32, codectest Int32) engine = MergeTree order by x; + +alter table t add projection x (select * order by codectest); + +insert into t values (1, 2); + +select * from merge('', 't'); + +drop table t; From 2a6b96f9e339e602c59968741741e57b1675bf52 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jul 2023 13:51:40 +0000 Subject: [PATCH 613/871] Automatic style fix --- .../test_memory_profiler_min_max_borders/test.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py index b768a442591..6ab971fa9c4 100644 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -23,5 +23,15 @@ def test_trace_boundaries_work(started_cluster): node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") node.query("SYSTEM FLUSH LOGS") - assert node.query("SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'") == "1\n" - assert node.query("SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)") == "0\n" + assert ( + node.query( + "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" + ) + == "0\n" + ) From 3acb6005f041051b7c00c48df5035843744a7e24 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jul 2023 17:08:01 +0200 Subject: [PATCH 614/871] Reduce the number of syscalls in FileCache::loadMetadata --- src/Interpreters/Cache/FileCache.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 91d1c63e832..42cc7b80a66 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -870,13 +870,12 @@ void FileCache::loadMetadata() } size_t total_size = 0; - for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; - key_prefix_it != fs::directory_iterator();) + for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; key_prefix_it != fs::directory_iterator(); + key_prefix_it++) { const fs::path key_prefix_directory = key_prefix_it->path(); - key_prefix_it++; - if (!fs::is_directory(key_prefix_directory)) + if (!key_prefix_it->is_directory()) { if (key_prefix_directory.filename() != "status") { @@ -887,19 +886,19 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_prefix_directory)) + fs::directory_iterator key_it{key_prefix_directory}; + if (key_it == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string()); fs::remove(key_prefix_directory); continue; } - for (fs::directory_iterator key_it{key_prefix_directory}; key_it != fs::directory_iterator();) + for (/* key_it already initialized to verify emptiness */; key_it != fs::directory_iterator(); key_it++) { const fs::path key_directory = key_it->path(); - ++key_it; - if (!fs::is_directory(key_directory)) + if (!key_it->is_directory()) { LOG_DEBUG( log, @@ -908,7 +907,7 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_directory)) + if (fs::directory_iterator{key_directory} == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string()); fs::remove(key_directory); From 5fb5ba71edbaf664045871b0fc8d6d5d6f5f45e6 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 15:40:53 +0000 Subject: [PATCH 615/871] 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 a1de14afc13..72706cb98ba 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 616/871] 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 841b646a126..bc8dbfa0e1f 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 617/871] 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 bc8dbfa0e1f..f191440442d 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 6c8d5ca0a554ecc4fee32269858797d139f3c02a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 21 Jul 2023 21:33:51 +0000 Subject: [PATCH 618/871] Fix: remove redundant distinct with views --- src/Interpreters/ActionsDAG.cpp | 18 +++++++++++---- ...x_remove_dedundant_distinct_view.reference | 13 +++++++++++ ...810_fix_remove_dedundant_distinct_view.sql | 22 +++++++++++++++++++ 3 files changed, 49 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference create mode 100644 tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 906875dd314..ce273e78ff3 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2511,11 +2511,21 @@ FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr /// find input node which refers to the output node /// consider only aliases on the path const auto * node = output_node; - while (node && node->type == ActionsDAG::ActionType::ALIAS) + while (node) { - /// alias has only one child - chassert(node->children.size() == 1); - node = node->children.front(); + if (node->type == ActionsDAG::ActionType::ALIAS) + { + node = node->children.front(); + } + /// materiailze can occure when dealing with views, special case + /// TODO: not sure if it should be done here, looks too generic place + else if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "materialize") + { + chassert(node->children.size() == 1); + node = node->children.front(); + } + else + break; } if (node && node->type == ActionsDAG::ActionType::INPUT) index.emplace(output_node->result_name, node); diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference new file mode 100644 index 00000000000..01f14f82e94 --- /dev/null +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference @@ -0,0 +1,13 @@ +-- { echoOn } +set query_plan_remove_redundant_distinct=1; +-- DISTINCT has to be removed since the view already has DISTINCT on the same column +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM tab_v +) +WHERE explain ILIKE '%distinct%'; +2 +SELECT DISTINCT x FROM tab_v; +2 +1 diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql new file mode 100644 index 00000000000..99fc24dae8b --- /dev/null +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -0,0 +1,22 @@ +set allow_experimental_analyzer=1; + +drop table if exists tab_v; +drop table if exists tab; +create table tab (x UInt64, y UInt64) engine MergeTree() order by (x, y); +insert into tab values(1, 1); +insert into tab values(1, 2); +insert into tab values(2, 1); + +create view tab_v as select distinct(x) from tab; + +-- { echoOn } +set query_plan_remove_redundant_distinct=1; +-- DISTINCT has to be removed since the view already has DISTINCT on the same column +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM tab_v +) +WHERE explain ILIKE '%distinct%'; + +SELECT DISTINCT x FROM tab_v; From c6ffc9f266f1bb8a667a3d5beff9bd47a288ef74 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 01:05:39 +0300 Subject: [PATCH 619/871] Update 02815_fix_not_found_constants_col_in_block.sql --- .../0_stateless/02815_fix_not_found_constants_col_in_block.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql index c56d59c72d6..fa784cf12e3 100644 --- a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -3,3 +3,4 @@ CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingM insert into t0 values (19000, 1); select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; +DROP TABLE t0; From 687cbc57bba42a67b62b1b717e51a5be7e14b733 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 21 Jul 2023 22:15:02 +0000 Subject: [PATCH 620/871] Fix typo --- src/Interpreters/ActionsDAG.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index ce273e78ff3..284c42b658a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2517,7 +2517,7 @@ FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr { node = node->children.front(); } - /// materiailze can occure when dealing with views, special case + /// materiailze() function can occur when dealing with views /// TODO: not sure if it should be done here, looks too generic place else if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "materialize") { From 5ec63c782c6bdd62705f26cc9b09e8a640ca9da8 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 00:15:05 +0200 Subject: [PATCH 621/871] Fixed inserting into Buffer engine by not throwing exception from DatabaseCatalog::tryGetTable() when database name is empty --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- ...rentDatabase_for_table_functions.reference | 17 +++++++++++++++++ ...14_currentDatabase_for_table_functions.sql | 19 +++++++++++++++++++ 3 files changed, 38 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference create mode 100644 tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index e0b6348ed3c..f9ed2c0d5ca 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - auto it = databases.find(table_id.getDatabaseName()); + // hasDatabase() to avod getDatabaseName() throwing exception if database is empty. + auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { if (exception) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference new file mode 100644 index 00000000000..7ff95106d3d --- /dev/null +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference @@ -0,0 +1,17 @@ +-- Based on https://github.com/ClickHouse/ClickHouse/issues/52436 +-- Test that inserts performed via Buffer table engine land into destination table. +-- { echoOn } + +DROP TABLE IF EXISTS null_table; +DROP TABLE IF EXISTS null_table_buffer; +DROP TABLE IF EXISTS null_mv; +DROP VIEW IF EXISTS number_view; +CREATE TABLE null_table (number UInt64) ENGINE = Null; +CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; +CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; +CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); +INSERT INTO null_table_buffer VALUES (1); +SELECT sleep(3) FORMAT Null; +-- Insert about should've landed into `null_mv` +SELECT count() FROM null_mv; +1 diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql new file mode 100644 index 00000000000..74b5cf5f432 --- /dev/null +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql @@ -0,0 +1,19 @@ +-- Based on https://github.com/ClickHouse/ClickHouse/issues/52436 +-- Test that inserts performed via Buffer table engine land into destination table. +-- { echoOn } + +DROP TABLE IF EXISTS null_table; +DROP TABLE IF EXISTS null_table_buffer; +DROP TABLE IF EXISTS null_mv; +DROP VIEW IF EXISTS number_view; + +CREATE TABLE null_table (number UInt64) ENGINE = Null; +CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; +CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; + +CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); +INSERT INTO null_table_buffer VALUES (1); +SELECT sleep(3) FORMAT Null; + +-- Insert about should've landed into `null_mv` +SELECT count() FROM null_mv; From 3cc7f137bd5f7623280a22b4a32d9a3169dcdfea Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 22 Jul 2023 00:45:12 +0200 Subject: [PATCH 622/871] add tests with connection reset by peer --- .../integration/helpers/s3_mocks/broken_s3.py | 270 +++++++++++------- .../test_checking_s3_blobs_paranoid/test.py | 214 +++++++++++++- tests/integration/test_merge_tree_s3/test.py | 6 +- 3 files changed, 388 insertions(+), 102 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 8ff4f9e9203..0c794ae1ad4 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -7,11 +7,18 @@ import urllib.parse import http.server import socketserver import string +import socket +import struct INF_COUNT = 100000000 +def _and_then(value, func): + assert callable(func) + return None if value is None else func(value) + + class MockControl: def __init__(self, cluster, container, port): self._cluster = cluster @@ -30,8 +37,8 @@ class MockControl: ) assert response == "OK", response - def setup_error_at_object_upload(self, count=None, after=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_object_upload?nothing=1" + def setup_action(self, when, count=None, after=None, action="error_500", action_args=None): + url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: url += f"&count={count}" @@ -39,25 +46,12 @@ class MockControl: if after is not None: url += f"&after={after}" - response = self._cluster.exec_in_container( - self._cluster.get_container_id(self._container), - [ - "curl", - "-s", - url, - ], - nothrow=True, - ) - assert response == "OK", response + if action is not None: + url += f"&action={action}" - def setup_error_at_part_upload(self, count=None, after=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_part_upload?nothing=1" - - if count is not None: - url += f"&count={count}" - - if after is not None: - url += f"&after={after}" + if action_args is not None: + for x in action_args: + url += f"&action_args={x}" response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), @@ -70,22 +64,14 @@ class MockControl: ) assert response == "OK", response - def setup_error_at_create_multi_part_upload(self, count=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_create_multi_part_upload" + def setup_at_object_upload(self, **kwargs): + self.setup_action("at_object_upload", **kwargs) - if count is not None: - url += f"?count={count}" + def setup_at_part_upload(self, **kwargs): + self.setup_action("at_part_upload", **kwargs) - response = self._cluster.exec_in_container( - self._cluster.get_container_id(self._container), - [ - "curl", - "-s", - url, - ], - nothrow=True, - ) - assert response == "OK", response + def setup_at_create_multi_part_upload(self, **kwargs): + self.setup_action("at_create_multi_part_upload", **kwargs) def setup_fake_puts(self, part_length): response = self._cluster.exec_in_container( @@ -166,13 +152,82 @@ class _ServerRuntime: return _runtime.slow_put.timeout return None + class Expected500ErrorAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "ExpectedError" + "mock s3 injected error" + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(data) + + class RedirectAction: + def __init__(self, host="localhost", port=1): + self.dst_host = _and_then(host, str) + self.dst_port = _and_then(port, int) + + def inject_error(self, request_handler): + request_handler.redirect(host=self.dst_host, port=self.dst_port) + + class ConnectionResetByPeerAction: + def __init__(self, with_partial_data=None): + self.partial_data = "" + if with_partial_data is not None: + self.partial_data = ( + '\n' + "\n" + ) + + def inject_error(self, request_handler): + request_handler.read_all_input() + + if self.partial_data: + request_handler.send_response(200) + request_handler.send_header("Content-Type", "text/xml") + request_handler.send_header("Content-Length", 10000) + request_handler.end_headers() + request_handler.wfile.write(bytes(self.partial_data, "UTF-8")) + + time.sleep(1) + request_handler.connection.setsockopt( + socket.SOL_SOCKET, + socket.SO_LINGER, + struct.pack('ii', 1, 0) + ) + request_handler.connection.close() + + class ConnectionRefusedAction(RedirectAction): + pass + class CountAfter: - def __init__(self, count_=None, after_=None): + def __init__(self, count_=None, after_=None, action_=None, action_args_=[]): self.count = count_ if count_ is not None else INF_COUNT self.after = after_ if after_ is not None else 0 + self.action = action_ + self.action_args = action_args_ + if self.action == "connection_refused": + self.error_handler = _ServerRuntime.ConnectionRefusedAction() + elif self.action == "connection_reset_by_peer": + self.error_handler = _ServerRuntime.ConnectionResetByPeerAction(*self.action_args) + elif self.action == "redirect_to": + self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) + else: + self.error_handler = _ServerRuntime.Expected500ErrorAction() + + @staticmethod + def from_cgi_params(params): + return _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), + action_=params.get("action", [None])[0], + action_args_=params.get("action_args", []), + ) def __str__(self): - return f"count:{self.count} after:{self.after}" + return f"count:{self.count} after:{self.after} action:{self.action} action_args:{self.action_args}" def has_effect(self): if self.after: @@ -183,15 +238,18 @@ class _ServerRuntime: return True return False + def inject_error(self, request_handler): + self.error_handler.inject_error(request_handler) + def __init__(self): self.lock = threading.Lock() - self.error_at_part_upload = None - self.error_at_object_upload = None + self.at_part_upload = None + self.at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None self.fake_multipart_upload = None - self.error_at_create_multi_part_upload = None + self.at_create_multi_part_upload = None def register_fake_upload(self, upload_id, key): with self.lock: @@ -205,23 +263,18 @@ class _ServerRuntime: def reset(self): with self.lock: - self.error_at_part_upload = None - self.error_at_object_upload = None + self.at_part_upload = None + self.at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None self.fake_multipart_upload = None - self.error_at_create_multi_part_upload = None + self.at_create_multi_part_upload = None _runtime = _ServerRuntime() -def _and_then(value, func): - assert callable(func) - return None if value is None else func(value) - - def get_random_string(length): # choose from all lowercase letter letters = string.ascii_lowercase @@ -239,7 +292,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def _ping(self): self._ok() - def _read_out(self): + def read_all_input(self): content_length = int(self.headers.get("Content-Length", 0)) to_read = content_length while to_read > 0: @@ -250,36 +303,38 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): str(self.rfile.read(size)) to_read -= size - def _redirect(self): - self._read_out() + def redirect(self, host=None, port=None): + if host is None and port is None: + host = self.server.upstream_host + port = self.server.upstream_port + + self.read_all_input() self.send_response(307) url = ( - f"http://{self.server.upstream_host}:{self.server.upstream_port}{self.path}" + f"http://{host}:{port}{self.path}" ) + self.log_message("redirect to %s", url) self.send_header("Location", url) self.end_headers() self.wfile.write(b"Redirected") - def _error(self, data): - self._read_out() + def write_error(self, data, content_length=None): + if content_length is None: + content_length = len(data) + self.log_message("write_error %s", data) + self.read_all_input() self.send_response(500) self.send_header("Content-Type", "text/xml") + self.send_header("Content-Length", str(content_length)) self.end_headers() - self.wfile.write(bytes(data, "UTF-8")) - - def _error_expected_500(self): - self._error( - '' - "" - "ExpectedError" - "mock s3 injected error" - "txfbd566d03042474888193-00608d7537" - "" - ) + if data: + self.wfile.write(bytes(data, "UTF-8")) def _fake_put_ok(self): - self._read_out() + self.log_message("fake put") + + self.read_all_input() self.send_response(200) self.send_header("Content-Type", "text/xml") @@ -288,7 +343,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.end_headers() def _fake_uploads(self, path, upload_id): - self._read_out() + self.read_all_input() parts = [x for x in path.split("/") if x] bucket = parts[0] @@ -310,7 +365,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.wfile.write(bytes(data, "UTF-8")) def _fake_post_ok(self, path): - self._read_out() + self.read_all_input() parts = [x for x in path.split("/") if x] bucket = parts[0] @@ -338,22 +393,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): path = [x for x in parts.path.split("/") if x] assert path[0] == "mock_settings", path if len(path) < 2: - return self._error("_mock_settings: wrong command") + return self.write_error("_mock_settings: wrong command") - if path[1] == "error_at_part_upload": + if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_part_upload = _ServerRuntime.CountAfter( - count_=_and_then(params.get("count", [None])[0], int), - after_=_and_then(params.get("after", [None])[0], int), - ) + _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_part_upload %s", _runtime.at_part_upload) return self._ok() - if path[1] == "error_at_object_upload": + if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_object_upload = _ServerRuntime.CountAfter( - count_=_and_then(params.get("count", [None])[0], int), - after_=_and_then(params.get("after", [None])[0], int), - ) + _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() if path[1] == "fake_puts": @@ -361,6 +412,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) + self.log_message("set fake_puts %s", _runtime.fake_put_when_length_bigger) return self._ok() if path[1] == "slow_put": @@ -376,20 +428,21 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "setup_fake_multpartuploads": _runtime.fake_multipart_upload = True + self.log_message("set setup_fake_multpartuploads") return self._ok() - if path[1] == "error_at_create_multi_part_upload": + if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_create_multi_part_upload = int( - params.get("count", [INF_COUNT])[0] - ) + _runtime.at_create_multi_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_create_multi_part_upload %s", _runtime.at_create_multi_part_upload) return self._ok() if path[1] == "reset": _runtime.reset() + self.log_message("reset") return self._ok() - return self._error("_mock_settings: wrong command") + return self.write_error("_mock_settings: wrong command") def do_GET(self): if self.path == "/": @@ -398,7 +451,8 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if self.path.startswith("/mock_settings"): return self._mock_settings() - return self._redirect() + self.log_message("get redirect") + return self.redirect() def do_PUT(self): content_length = int(self.headers.get("Content-Length", 0)) @@ -414,30 +468,52 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): upload_id = params.get("uploadId", [None])[0] if upload_id is not None: - if _runtime.error_at_part_upload is not None: - if _runtime.error_at_part_upload.has_effect(): - return self._error_expected_500() + if _runtime.at_part_upload is not None: + self.log_message( + "put error_at_object_upload %s, %s, %s", + _runtime.at_part_upload, + upload_id, + parts, + ) + + if _runtime.at_part_upload.has_effect(): + return _runtime.at_part_upload.inject_error(self) if _runtime.fake_multipart_upload: if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_put_ok() else: - if _runtime.error_at_object_upload is not None: - if _runtime.error_at_object_upload.has_effect(): - return self._error_expected_500() + if _runtime.at_object_upload is not None: + if _runtime.at_object_upload.has_effect(): + self.log_message( + "put error_at_object_upload %s, %s, %s", + _runtime.at_object_upload, + parts, + ) + return _runtime.at_object_upload.inject_error(self) if _runtime.fake_put_when_length_bigger is not None: if content_length > _runtime.fake_put_when_length_bigger: + self.log_message( + "put fake_put_when_length_bigger %s, %s, %s", + _runtime.fake_put_when_length_bigger, + content_length, + parts, + ) return self._fake_put_ok() - return self._redirect() + self.log_message( + "put redirect %s", + parts, + ) + return self.redirect() def do_POST(self): parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=True) uploads = params.get("uploads", [None])[0] if uploads is not None: - if _runtime.error_at_create_multi_part_upload: - _runtime.error_at_create_multi_part_upload -= 1 - return self._error_expected_500() + if _runtime.at_create_multi_part_upload is not None: + if _runtime.at_create_multi_part_upload.has_effect(): + return _runtime.at_create_multi_part_upload.inject_error(self) if _runtime.fake_multipart_upload: upload_id = get_random_string(5) @@ -448,13 +524,13 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) - return self._redirect() + return self.redirect() def do_HEAD(self): - self._redirect() + self.redirect() def do_DELETE(self): - self._redirect() + self.redirect() class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index a80ad93d53d..b6b598c4557 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -41,6 +41,11 @@ def broken_s3(init_broken_s3): yield init_broken_s3 +@pytest.fixture(scope="module") +def init_connection_reset_by_peer(cluster): + yield start_s3_mock(cluster, "connection_reset_by_peer", "8084") + + def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] @@ -91,7 +96,7 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression): node = cluster.instances["node"] - broken_s3.setup_error_at_create_multi_part_upload() + broken_s3.setup_at_create_multi_part_upload() insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" error = node.query_and_get_error( @@ -134,7 +139,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( node = cluster.instances["node"] broken_s3.setup_fake_multpartuploads() - broken_s3.setup_error_at_part_upload(count=1, after=2) + broken_s3.setup_at_part_upload(count=1, after=2) insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" error = node.query_and_get_error( @@ -165,3 +170,208 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert count_create_multi_part_uploads == 1 assert count_upload_parts >= 2 assert count_s3_errors >= 2 + + +def test_when_s3_connection_refused_at_write_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload(count=3, after=2, action="connection_refused") + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error, error + + +def test_when_s3_connection_reset_by_peer_at_write_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="connection_reset_by_peer", + ) + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + + +def test_when_s3_connection_reset_by_peer_at_read_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_create_multi_part_upload( + count=3, + after=0, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_create_multi_part_upload( + count=1000, + after=0, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f754bc905bf..d7c267eed50 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -783,9 +783,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - broken_s3.setup_error_at_object_upload() + broken_s3.setup_at_object_upload() broken_s3.setup_fake_multpartuploads() - broken_s3.setup_error_at_part_upload() + broken_s3.setup_at_part_upload() node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -828,7 +828,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): settings={"materialize_ttl_after_modify": 0}, ) - broken_s3.setup_error_at_object_upload(count=1, after=1) + broken_s3.setup_at_object_upload(count=1, after=1) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") From ae8f511ac5ffc6191394dd9fbfed9a0b082102e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:27:07 +0200 Subject: [PATCH 623/871] 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 00000000000..b0990ca3a60 --- /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 4f3d42096c3..dc79e5d8723 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 9a5aed35e24a9aa4d7de71971665449cf344f917 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:33:44 +0200 Subject: [PATCH 624/871] Add a note about potential caveats for the "session_timezone" setting --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfcb56729d2..f267fa15276 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -775,7 +775,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ 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(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, 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. From e68234a231bf234d60ccfa262ca5a2374fb4f98a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:45:50 +0300 Subject: [PATCH 625/871] Revert "Re-add SipHash keyed functions" --- .../sql-reference/functions/hash-functions.md | 8 +- src/Functions/FunctionsHashing.h | 329 +++++------------- src/Functions/FunctionsHashingMisc.cpp | 5 - .../0_stateless/02534_keyed_siphash.reference | 37 -- .../0_stateless/02534_keyed_siphash.sql | 61 +--- .../02552_siphash128_reference.reference | 151 -------- .../02552_siphash128_reference.sql | 253 -------------- 7 files changed, 99 insertions(+), 745 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 556fe622c27..06097d92480 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#hash_functions-siphash64} +## sipHash64 (#hash_functions-siphash64) Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 82944630b10..279294b367c 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -79,51 +79,28 @@ namespace impl UInt64 key1 = 0; }; - struct SipHashKeyColumns + static SipHashKey parseSipHashKey(const ColumnWithTypeAndName & key) { - ColumnPtr key0; - ColumnPtr key1; - bool is_const; + SipHashKey ret{}; - size_t size() const - { - assert(key0 && key1); - assert(key0->size() == key1->size()); - return key0->size(); - } - SipHashKey getKey(size_t i) const - { - if (is_const) - i = 0; - const auto & key0data = assert_cast(*key0).getData(); - const auto & key1data = assert_cast(*key1).getData(); - return {key0data[i], key1data[i]}; - } - }; - - static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key) - { - const ColumnTuple * tuple = nullptr; - const auto * column = key.column.get(); - bool is_const = false; - if (isColumnConst(*column)) - { - is_const = true; - tuple = checkAndGetColumnConstData(column); - } - else - tuple = checkAndGetColumn(column); + const auto * tuple = checkAndGetColumn(key.column.get()); if (!tuple) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple"); + if (tuple->tupleSize() != 2) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); - SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const}; - assert(ret.key0); - if (!checkColumn(*ret.key0)) + if (tuple->empty()) + return ret; + + if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) + ret.key0 = key0col->get64(0); + else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); - assert(ret.key1); - if (!checkColumn(*ret.key1)) + + if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) + ret.key1 = key1col->get64(0); + else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); return ret; @@ -352,10 +329,8 @@ struct SipHash64KeyedImpl static constexpr auto name = "sipHash64Keyed"; using ReturnType = UInt64; using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); } @@ -396,10 +371,8 @@ struct SipHash128KeyedImpl static constexpr auto name = "sipHash128Keyed"; using ReturnType = UInt128; using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); } @@ -425,43 +398,13 @@ struct SipHash128ReferenceImpl using ReturnType = UInt128; - static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); } static constexpr bool use_int_hash_for_pods = false; }; -struct SipHash128ReferenceKeyedImpl -{ - static constexpr auto name = "sipHash128ReferenceKeyed"; - using ReturnType = UInt128; - using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } - - static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) - { - return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); - } - - static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) - { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - UInt128 tmp; - reverseMemcpy(&tmp, &h1, sizeof(UInt128)); - h1 = tmp; - reverseMemcpy(&tmp, &h2, sizeof(UInt128)); - h2 = tmp; -#endif - UInt128 hashes[] = {h1, h2}; - return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); - } - - static constexpr bool use_int_hash_for_pods = false; -}; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. @@ -1080,7 +1023,7 @@ private: DECLARE_MULTITARGET_CODE( -template +template class FunctionAnyHash : public IFunction { public: @@ -1090,12 +1033,9 @@ private: using ToType = typename Impl::ReturnType; template - void executeIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1104,9 +1044,6 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); if constexpr (Impl::use_int_hash_for_pods) { @@ -1140,14 +1077,6 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeIntType(key_cols, full_column.get(), vec_to); - } - } auto value = col_from_const->template getValue(); ToType hash; @@ -1178,15 +1107,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1194,12 +1116,9 @@ private: } template - void executeBigIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeBigIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1208,9 +1127,6 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); if constexpr (std::endian::native == std::endian::little) hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); else @@ -1227,14 +1143,6 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeBigIntType(key_cols, full_column.get(), vec_to); - } - } auto value = col_from_const->template getValue(); ToType hash; @@ -1250,15 +1158,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1266,16 +1167,10 @@ private: } template - void executeGeneric(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeGeneric(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); for (size_t i = 0, size = column->size(); i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); StringRef bytes = column->getDataAt(i); const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) @@ -1286,11 +1181,8 @@ private: } template - void executeString(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeString(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColumnString * col_from = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from->getChars(); @@ -1300,9 +1192,6 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); @@ -1323,9 +1212,6 @@ private: for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) vec_to[i] = hash; @@ -1335,14 +1221,6 @@ private: } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeString(key_cols, full_column.get(), vec_to); - } - } String value = col_from_const->getValue(); const ToType hash = apply(key, value.data(), value.size()); const size_t size = vec_to.size(); @@ -1350,15 +1228,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1366,7 +1237,7 @@ private: } template - void executeArray(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const { const IDataType * nested_type = typeid_cast(*type).getNestedType().get(); @@ -1378,19 +1249,13 @@ private: typename ColumnVector::Container vec_temp(nested_size); bool nested_is_first = true; - executeForArgument(key_cols, nested_type, nested_column, vec_temp, nested_is_first); + executeForArgument(key, nested_type, nested_column, vec_temp, nested_is_first); const size_t size = offsets.size(); ColumnArray::Offset current_offset = 0; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); ColumnArray::Offset next_offset = offsets[i]; ToType hash; @@ -1414,7 +1279,7 @@ private: { /// NOTE: here, of course, you can do without the materialization of the column. ColumnPtr full_column = col_from_const->convertToFullColumn(); - executeArray(key_cols, type, full_column.get(), vec_to); + executeArray(key, type, full_column.get(), vec_to); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1422,7 +1287,7 @@ private: } template - void executeAny(const KeyColumnsType & key_cols, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const + void executeAny(const KeyType & key, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const { WhichDataType which(from_type); @@ -1430,45 +1295,40 @@ private: throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", icolumn->getName(), icolumn->size(), vec_to.size(), getName()); - if constexpr (Keyed) - if ((!key_cols.is_const && key_cols.size() != vec_to.size()) - || (key_cols.is_const && key_cols.size() != 1)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName()); - - if (which.isUInt8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isUInt256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isInt8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isInt256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isUUID()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isIPv4()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isIPv6()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isEnum8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isEnum16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDate()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDate32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDateTime()) executeIntType(key_cols, icolumn, vec_to); + if (which.isUInt8()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt16()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt32()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt64()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isUInt256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isInt8()) executeIntType(key, icolumn, vec_to); + else if (which.isInt16()) executeIntType(key, icolumn, vec_to); + else if (which.isInt32()) executeIntType(key, icolumn, vec_to); + else if (which.isInt64()) executeIntType(key, icolumn, vec_to); + else if (which.isInt128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isInt256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isUUID()) executeBigIntType(key, icolumn, vec_to); + else if (which.isIPv4()) executeIntType(key, icolumn, vec_to); + else if (which.isIPv6()) executeBigIntType(key, icolumn, vec_to); + else if (which.isEnum8()) executeIntType(key, icolumn, vec_to); + else if (which.isEnum16()) executeIntType(key, icolumn, vec_to); + else if (which.isDate()) executeIntType(key, icolumn, vec_to); + else if (which.isDate32()) executeIntType(key, icolumn, vec_to); + else if (which.isDateTime()) executeIntType(key, icolumn, vec_to); /// TODO: executeIntType() for Decimal32/64 leads to incompatible result - else if (which.isDecimal32()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal64()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isFloat32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isFloat64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isString()) executeString(key_cols, icolumn, vec_to); - else if (which.isFixedString()) executeString(key_cols, icolumn, vec_to); - else if (which.isArray()) executeArray(key_cols, from_type, icolumn, vec_to); - else executeGeneric(key_cols, icolumn, vec_to); + else if (which.isDecimal32()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal64()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isFloat32()) executeIntType(key, icolumn, vec_to); + else if (which.isFloat64()) executeIntType(key, icolumn, vec_to); + else if (which.isString()) executeString(key, icolumn, vec_to); + else if (which.isFixedString()) executeString(key, icolumn, vec_to); + else if (which.isArray()) executeArray(key, from_type, icolumn, vec_to); + else executeGeneric(key, icolumn, vec_to); } - void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const + void executeForArgument(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. if (const ColumnTuple * tuple = typeid_cast(column)) @@ -1477,7 +1337,7 @@ private: const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) - executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); + executeForArgument(key, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData(column)) { @@ -1487,24 +1347,24 @@ private: for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); - executeForArgument(key_cols, tuple_types[i].get(), tmp.get(), vec_to, is_first); + executeForArgument(key, tuple_types[i].get(), tmp.get(), vec_to, is_first); } } else if (const auto * map = checkAndGetColumn(column)) { const auto & type_map = assert_cast(*type); - executeForArgument(key_cols, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); + executeForArgument(key, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); } else if (const auto * const_map = checkAndGetColumnConst(column)) { - executeForArgument(key_cols, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); + executeForArgument(key, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); } else { if (is_first) - executeAny(key_cols, type, column, vec_to); + executeAny(key, type, column, vec_to); else - executeAny(key_cols, type, column, vec_to); + executeAny(key, type, column, vec_to); } is_first = false; @@ -1535,33 +1395,30 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count != 0) + typename ColumnVector::Container & vec_to = col_to->getData(); + + /// If using a "keyed" algorithm, the first argument is the key and + /// the data starts from the second argument. + /// Otherwise there is no key and all arguments are interpreted as data. + constexpr size_t first_data_argument = Keyed; + + if (arguments.size() <= first_data_argument) { - typename ColumnVector::Container & vec_to = col_to->getData(); + /// Return a fixed random-looking magic number when input is empty + vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); + } - /// If using a "keyed" algorithm, the first argument is the key and - /// the data starts from the second argument. - /// Otherwise there is no key and all arguments are interpreted as data. - constexpr size_t first_data_argument = Keyed; + KeyType key{}; + if constexpr (Keyed) + if (!arguments.empty()) + key = Impl::parseKey(arguments[0]); - if (arguments.size() <= first_data_argument) - { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } - - KeyColumnsType key_cols{}; - if constexpr (Keyed) - if (!arguments.empty()) - key_cols = Impl::parseKeyColumns(arguments[0]); - - /// The function supports arbitrary number of arguments of arbitrary types. - bool is_first_argument = true; - for (size_t i = first_data_argument; i < arguments.size(); ++i) - { - const auto & col = arguments[i]; - executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); - } + /// The function supports arbitrary number of arguments of arbitrary types. + bool is_first_argument = true; + for (size_t i = first_data_argument; i < arguments.size(); ++i) + { + const auto & col = arguments[i]; + executeForArgument(key, col.type.get(), col.column.get(), vec_to, is_first_argument); } if constexpr (std::is_same_v) /// backward-compatible @@ -1593,19 +1450,17 @@ public: ) // DECLARE_MULTITARGET_CODE -template -class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: explicit FunctionAnyHash(ContextPtr context) : selector(context) { - selector - .registerImplementation>(); + selector.registerImplementation>(); #if USE_MULTITARGET_CODE - selector.registerImplementation>(); - selector - .registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); #endif } @@ -1841,7 +1696,7 @@ struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; using FunctionSipHash64 = FunctionAnyHash; -using FunctionSipHash64Keyed = FunctionAnyHash; +using FunctionSipHash64Keyed = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL @@ -1855,10 +1710,8 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; -using FunctionSipHash128ReferenceKeyed - = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index f56568b2508..56c3c1ed00c 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing) .examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}}, .categories{"Hash"} }); - factory.registerFunction(FunctionDocumentation{ - .description = "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument " - "instead of using a fixed key.", - .examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}}, - .categories{"Hash"}}); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index a9f724365a8..ccc514e7ea2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,40 +197,3 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 -86AE90BB6A238D3F6221457630142C9B -86AE90BB6A238D3F6221457630142C9B -Check memsan bug -18096612095653370192 -20AF99D3A87829E0 -12489502208762728797 -Check const columns -15080046610211022027 -15080046610211022027 -15080046610211022027 -15080046610211022027 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -Check multiple keys as tuple from a table -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check multiple keys as separate ints from a table -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check constant key and data from a table -11862823756610506724 -11862823756610506724 -86AE90BB6A238D3F6221457630142C9B -86AE90BB6A238D3F6221457630142C9B -Check multiple keys as separate ints from a table with constant data -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check asan bug -0 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 4f3ae7d62bd..900b99f548a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash64Keyed((0, 0), '1'); -- { serverError 48 } +select sipHash128Keyed((0, 0), '1'); -- { serverError 48 } +select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 } select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; @@ -280,57 +280,4 @@ INSERT INTO tab VALUES ((2, 2), 4); -- these two statements must produce the same result SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; -SELECT hex(sipHash128Keyed(key, val)) FROM tab; -SELECT hex(sipHash128Keyed(key, 4::UInt64)) FROM tab; DROP TABLE tab; - -SELECT 'Check memsan bug'; -SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); -SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); -SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); - -SELECT 'Check const columns'; -DROP TABLE IF EXISTS sipHashKeyed_test; -CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); -SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), 1, 'test')); -SELECT hex(sipHash128(tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; -DROP TABLE sipHashKeyed_test; - -SELECT 'Check multiple keys as tuple from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); -INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); -SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; -SELECT hex(sipHash128Keyed(key, val)) FROM sipHashKeyed_keys ORDER by key; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); -INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); -SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; -SELECT hex(sipHash128Keyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; -SELECT 'Check constant key and data from a table'; -SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; -SELECT hex(sipHash128Keyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table with constant data'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2); -INSERT INTO sipHashKeyed_keys VALUES (4, 4); -SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; -SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check asan bug'; -SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index ece9f6a4615..d00491fd7e5 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1,152 +1 @@ -A3817F04BA25A8E66DF67214C7550293 -DA87C1D86B99AF44347659119B22FC45 -8177228DA4A45DC7FCA38BDEF60AFFE4 -9C70B60C5267A94E5F33B6B02985ED51 -F88164C12D9C8FAF7D0F6E7C7BCD5579 -1368875980776F8854527A07690E9627 -14EECA338B208613485EA0308FD7A15E -A1F1EBBED8DBC153C0B84AA61FF08239 -3B62A9BA6258F5610F83E264F31497B4 -264499060AD9BAABC47F8B02BB6D71ED -00110DC378146956C95447D3F3D0FBBA -0151C568386B6677A2B4DC6F81E5DC18 -D626B266905EF35882634DF68532C125 -9869E247E9C08B10D029934FC4B952F7 -31FCEFAC66D7DE9C7EC7485FE4494902 -5493E99933B0A8117E08EC0F97CFC3D9 -6EE2A4CA67B054BBFD3315BF85230577 -473D06E8738DB89854C066C47AE47740 -A426E5E423BF4885294DA481FEAEF723 -78017731CF65FAB074D5208952512EB1 -9E25FC833F2290733E9344A5E83839EB -568E495ABE525A218A2214CD3E071D12 -4A29B54552D16B9A469C10528EFF0AAE -C9D184DDD5A9F5E0CF8CE29A9ABF691C -2DB479AE78BD50D8882A8A178A6132AD -8ECE5F042D5E447B5051B9EACB8D8F6F -9C0B53B4B3C307E87EAEE08678141F66 -ABF248AF69A6EAE4BFD3EB2F129EEB94 -0664DA1668574B88B935F3027358AEF4 -AA4B9DC4BF337DE90CD4FD3C467C6AB7 -EA5C7F471FAF6BDE2B1AD7D4686D2287 -2939B0183223FAFC1723DE4F52C43D35 -7C3956CA5EEAFC3E363E9D556546EB68 -77C6077146F01C32B6B69D5F4EA9FFCF -37A6986CB8847EDF0925F0F1309B54DE -A705F0E69DA9A8F907241A2E923C8CC8 -3DC47D1F29C448461E9E76ED904F6711 -0D62BF01E6FC0E1A0D3C4751C5D3692B -8C03468BCA7C669EE4FD5E084BBEE7B5 -528A5BB93BAF2C9C4473CCE5D0D22BD9 -DF6A301E95C95DAD97AE0CC8C6913BD8 -801189902C857F39E73591285E70B6DB -E617346AC9C231BB3650AE34CCCA0C5B -27D93437EFB721AA401821DCEC5ADF89 -89237D9DED9C5E78D8B1C9B166CC7342 -4A6D8091BF5E7D651189FA94A250B14C -0E33F96055E7AE893FFC0E3DCF492902 -E61C432B720B19D18EC8D84BDC63151B -F7E5AEF549F782CF379055A608269B16 -438D030FD0B7A54FA837F2AD201A6403 -A590D3EE4FBF04E3247E0D27F286423F -5FE2C1A172FE93C4B15CD37CAEF9F538 -2C97325CBD06B36EB2133DD08B3A017C -92C814227A6BCA949FF0659F002AD39E -DCE850110BD8328CFBD50841D6911D87 -67F14984C7DA791248E32BB5922583DA -1938F2CF72D54EE97E94166FA91D2A36 -74481E9646ED49FE0F6224301604698E -57FCA5DE98A9D6D8006438D0583D8A1D -9FECDE1CEFDC1CBED4763674D9575359 -E3040C00EB28F15366CA73CBD872E740 -7697009A6A831DFECCA91C5993670F7A -5853542321F567A005D547A4F04759BD -5150D1772F50834A503E069A973FBD7C 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -Check bug with hashing of const integer values -E940B12600C844966162FF8FE7A16AAE -E940B12600C844966162FF8FE7A16AAE -Check memsan bug -1CE422FEE7BD8DE20000000000000000 -Check const columns -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -Check multiple keys as tuple from a table -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 -Check multiple keys as separate ints from a table -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 -Check constant key and data from a table -E940B12600C844966162FF8FE7A16AAE -E940B12600C844966162FF8FE7A16AAE -Check multiple keys as separate ints from a table with constant data -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index f7324ed0ee4..200954c3b57 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1,254 +1 @@ --- Test Vectors from the SipHash reference C implementation: --- Written by --- Jean-Philippe Aumasson --- Daniel J. Bernstein --- Released under CC0 --- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 - -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - '')); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); - --- CH tests -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); - -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } - SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; -SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; - -SELECT 'Check bug with hashing of const integer values'; -DROP TABLE IF EXISTS tab; -CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO tab VALUES ((2, 2), 4); --- these two statements must produce the same result -SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM tab; -SELECT hex(sipHash128ReferenceKeyed(key, 4::UInt64)) FROM tab; -DROP TABLE tab; - -SELECT 'Check memsan bug'; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); - -SELECT 'Check const columns'; -DROP TABLE IF EXISTS sipHashKeyed_test; -CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), 1, 'test')); -SELECT hex(sipHash128Reference(tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; -DROP TABLE sipHashKeyed_test; - -SELECT 'Check multiple keys as tuple from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); -INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); -SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM sipHashKeyed_keys ORDER by key; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); -INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); -SELECT hex(sipHash128ReferenceKeyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; -SELECT 'Check constant key and data from a table'; -SELECT hex(sipHash128ReferenceKeyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table with constant data'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2); -INSERT INTO sipHashKeyed_keys VALUES (4, 4); -SELECT hex(sipHash128ReferenceKeyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; -DROP TABLE sipHashKeyed_keys; From 7cc3372355d06dfc1184b3ebcd6d2164d179b7be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:30:09 +0200 Subject: [PATCH 626/871] Fix terrible trash --- src/Functions/FunctionsHashing.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 82944630b10..090d38fa73d 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1567,7 +1567,10 @@ public: if constexpr (std::is_same_v) /// backward-compatible { auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128)); - col_to_fixed_string->getChars() = std::move(*reinterpret_cast(&col_to->getData())); + const auto & data = col_to->getData(); + auto & chars = col_to_fixed_string->getChars(); + chars.resize(data.size() * sizeof(UInt128)); + memcpy(chars.data(), data.data(), data.size() * sizeof(UInt128)); return col_to_fixed_string; } @@ -1601,12 +1604,12 @@ public: { selector .registerImplementation>(); - +/* #if USE_MULTITARGET_CODE selector.registerImplementation>(); selector .registerImplementation>(); -#endif +#endif*/ } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override From 4c2dabddb6d697ba3744e48e07e09aeaf8fc59d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:31:38 +0200 Subject: [PATCH 627/871] Add a test --- tests/queries/0_stateless/02831_trash.reference | 2 ++ tests/queries/0_stateless/02831_trash.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02831_trash.reference create mode 100644 tests/queries/0_stateless/02831_trash.sql diff --git a/tests/queries/0_stateless/02831_trash.reference b/tests/queries/0_stateless/02831_trash.reference new file mode 100644 index 00000000000..e25f2e9e23f --- /dev/null +++ b/tests/queries/0_stateless/02831_trash.reference @@ -0,0 +1,2 @@ +2761631236 +1210084689 diff --git a/tests/queries/0_stateless/02831_trash.sql b/tests/queries/0_stateless/02831_trash.sql new file mode 100644 index 00000000000..600e2ad0695 --- /dev/null +++ b/tests/queries/0_stateless/02831_trash.sql @@ -0,0 +1,2 @@ +SELECT CRC32IEEE(sipHash128()); +SELECT CRC32(murmurHash3_128()); From d2b178536e1e5b6d85c917d3d26bbe2cff7594ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:38:16 +0200 Subject: [PATCH 628/871] Fix terrible trash --- src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 090d38fa73d..8f8715ec3f1 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1604,12 +1604,12 @@ public: { selector .registerImplementation>(); -/* + #if USE_MULTITARGET_CODE selector.registerImplementation>(); selector .registerImplementation>(); -#endif*/ +#endif } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override From 4d0b75ebdd1bb69e155b237768c7db7a22cb09cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:42:16 +0200 Subject: [PATCH 629/871] Remove hashid --- .gitmodules | 3 - contrib/CMakeLists.txt | 1 - contrib/hashidsxx | 1 - contrib/hashidsxx-cmake/CMakeLists.txt | 14 -- src/Core/Settings.h | 2 +- src/Functions/CMakeLists.txt | 1 - src/Functions/FunctionHashID.cpp | 12 -- src/Functions/FunctionHashID.h | 170 ------------------ .../0_stateless/02293_hashid.reference | 15 -- tests/queries/0_stateless/02293_hashid.sql | 16 -- ...new_functions_must_be_documented.reference | 1 - 11 files changed, 1 insertion(+), 235 deletions(-) delete mode 160000 contrib/hashidsxx delete mode 100644 contrib/hashidsxx-cmake/CMakeLists.txt delete mode 100644 src/Functions/FunctionHashID.cpp delete mode 100644 src/Functions/FunctionHashID.h delete mode 100644 tests/queries/0_stateless/02293_hashid.reference delete mode 100644 tests/queries/0_stateless/02293_hashid.sql diff --git a/.gitmodules b/.gitmodules index 151dc28c55b..ba71a8ae3a7 100644 --- a/.gitmodules +++ b/.gitmodules @@ -258,9 +258,6 @@ [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash -[submodule "contrib/hashidsxx"] - path = contrib/hashidsxx - url = https://github.com/schoentoon/hashidsxx [submodule "contrib/nats-io"] path = contrib/nats-io url = https://github.com/ClickHouse/nats.c diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2af468970f1..0f68c0cbc7c 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -164,7 +164,6 @@ add_contrib (libpq-cmake libpq) add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (datasketches-cpp-cmake datasketches-cpp) -add_contrib (hashidsxx-cmake hashidsxx) option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES}) if (ENABLE_NLP) diff --git a/contrib/hashidsxx b/contrib/hashidsxx deleted file mode 160000 index 783f6911ccf..00000000000 --- a/contrib/hashidsxx +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 783f6911ccfdaca83e3cfac084c4aad888a80cee diff --git a/contrib/hashidsxx-cmake/CMakeLists.txt b/contrib/hashidsxx-cmake/CMakeLists.txt deleted file mode 100644 index 17f3888bd94..00000000000 --- a/contrib/hashidsxx-cmake/CMakeLists.txt +++ /dev/null @@ -1,14 +0,0 @@ -set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/hashidsxx") - -set (SRCS - "${LIBRARY_DIR}/hashids.cpp" -) - -set (HDRS - "${LIBRARY_DIR}/hashids.h" -) - -add_library(_hashidsxx ${SRCS} ${HDRS}) -target_include_directories(_hashidsxx SYSTEM PUBLIC "${LIBRARY_DIR}") - -add_library(ch_contrib::hashidsxx ALIAS _hashidsxx) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfcb56729d2..bde51ae9971 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -761,7 +761,7 @@ class IColumn; /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ - M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ + M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 2f5c8a212f2..06436488050 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -21,7 +21,6 @@ list (APPEND PUBLIC_LIBS dbms ch_contrib::metrohash ch_contrib::murmurhash - ch_contrib::hashidsxx ch_contrib::morton_nd ) diff --git a/src/Functions/FunctionHashID.cpp b/src/Functions/FunctionHashID.cpp deleted file mode 100644 index 829b3d9d2f6..00000000000 --- a/src/Functions/FunctionHashID.cpp +++ /dev/null @@ -1,12 +0,0 @@ -#include "FunctionHashID.h" -#include - -namespace DB -{ - -REGISTER_FUNCTION(HashID) -{ - factory.registerFunction(); -} - -} diff --git a/src/Functions/FunctionHashID.h b/src/Functions/FunctionHashID.h deleted file mode 100644 index 680c3f6430b..00000000000 --- a/src/Functions/FunctionHashID.h +++ /dev/null @@ -1,170 +0,0 @@ -#pragma once - -#include "config.h" - -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SUPPORT_IS_DISABLED; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; -} - -// hashid(string, salt) -class FunctionHashID : public IFunction -{ -public: - static constexpr auto name = "hashid"; - - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettingsRef().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Hashing function '{}' is experimental. Set `allow_experimental_hash_functions` setting to enable it", name); - - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least one argument", getName()); - - const auto & id_col = arguments[0]; - if (!isUnsignedInteger(id_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument of function {} must be unsigned integer, got {}", - getName(), - arguments[0].type->getName()); - - if (arguments.size() > 1) - { - const auto & hash_col = arguments[1]; - if (!isString(hash_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Second argument of function {} must be String, got {}", - getName(), - arguments[1].type->getName()); - } - - if (arguments.size() > 2) - { - const auto & min_length_col = arguments[2]; - if (!isUInt8(min_length_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Third argument of function {} must be UInt8, got {}", - getName(), - arguments[2].type->getName()); - } - - if (arguments.size() > 3) - { - const auto & alphabet_col = arguments[3]; - if (!isString(alphabet_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Fourth argument of function {} must be String, got {}", - getName(), - arguments[3].type->getName()); - } - - if (arguments.size() > 4) - { - throw Exception( - ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, - "Function {} expect no more than four arguments (integer, salt, min_length, optional_alphabet), got {}", - getName(), - arguments.size()); - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - const auto & numcolumn = arguments[0].column; - - if (checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get()) - || checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get())) - { - std::string salt; - UInt8 min_length = 0; - std::string alphabet; - - if (arguments.size() >= 4) - { - const auto & alphabetcolumn = arguments[3].column; - if (const auto * alpha_col = checkAndGetColumnConst(alphabetcolumn.get())) - { - alphabet = alpha_col->getValue(); - if (alphabet.find('\0') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Custom alphabet must not contain null character"); - } - } - else - alphabet.assign(DEFAULT_ALPHABET); - - if (arguments.size() >= 3) - { - const auto & minlengthcolumn = arguments[2].column; - if (const auto * min_length_col = checkAndGetColumnConst(minlengthcolumn.get())) - min_length = min_length_col->getValue(); - } - - if (arguments.size() >= 2) - { - const auto & saltcolumn = arguments[1].column; - if (const auto * salt_col = checkAndGetColumnConst(saltcolumn.get())) - salt = salt_col->getValue(); - } - - hashidsxx::Hashids hash(salt, min_length, alphabet); - - auto col_res = ColumnString::create(); - - for (size_t i = 0; i < input_rows_count; ++i) - { - col_res->insert(hash.encode({numcolumn->getUInt(i)})); - } - - return col_res; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function hashid", - arguments[0].column->getName()); - } -}; - -} diff --git a/tests/queries/0_stateless/02293_hashid.reference b/tests/queries/0_stateless/02293_hashid.reference deleted file mode 100644 index dfc78349c05..00000000000 --- a/tests/queries/0_stateless/02293_hashid.reference +++ /dev/null @@ -1,15 +0,0 @@ -0 gY -1 jR -2 k5 -3 l5 -4 mO -0 pbgkmdljlpjoapne -1 akemglnjepjpodba -2 obmgndljgajpkeao -3 dldokmpjpgjgeanb -4 nkdlpgajngjnobme -YQrvD5XGvbx -Bm3zaOq7zbp -oV -oV -6b diff --git a/tests/queries/0_stateless/02293_hashid.sql b/tests/queries/0_stateless/02293_hashid.sql deleted file mode 100644 index 06af0b5e1d8..00000000000 --- a/tests/queries/0_stateless/02293_hashid.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Tags: no-upgrade-check -SET allow_experimental_hash_functions = 1; - -select number, hashid(number) from system.numbers limit 5; -select number, hashid(number, 's3cr3t', 16, 'abcdefghijklmnop') from system.numbers limit 5; -select hashid(1234567890123456, 's3cr3t'); -select hashid(1234567890123456, 's3cr3t2'); - -SELECT hashid(1, hashid(2)); -SELECT hashid(1, 'k5'); -SELECT hashid(1, 'k5_othersalt'); - --- https://github.com/ClickHouse/ClickHouse/issues/39672 -SELECT - JSONExtractRaw(257, NULL), - hashid(1024, if(rand() % 10, 'truetruetruetrue', NULL), 's3\0r3t'); -- {serverError 43} 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 fc00bfdadca..595ebb483d5 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 @@ -346,7 +346,6 @@ hasAny hasColumnInTable hasSubstr hasThreadFuzzer -hashid hex hiveHash hop From 5f4756fb33f754913f4ab8ddfa84c39739920f19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:54:58 +0200 Subject: [PATCH 630/871] Remove toDecimalString --- .../functions/type-conversion-functions.md | 38 --- .../functions/type-conversion-functions.md | 38 --- src/Functions/FunctionToDecimalString.cpp | 22 -- src/Functions/FunctionToDecimalString.h | 312 ------------------ src/IO/WriteHelpers.h | 44 +-- .../02676_to_decimal_string.reference | 21 -- .../0_stateless/02676_to_decimal_string.sql | 35 -- 7 files changed, 13 insertions(+), 497 deletions(-) delete mode 100644 src/Functions/FunctionToDecimalString.cpp delete mode 100644 src/Functions/FunctionToDecimalString.h delete mode 100644 tests/queries/0_stateless/02676_to_decimal_string.reference delete mode 100644 tests/queries/0_stateless/02676_to_decimal_string.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 36f40b37238..c2bd525c483 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -945,44 +945,6 @@ Result: └────────────┴───────┘ ``` -## toDecimalString - -Converts a numeric value to String with the number of fractional digits in the output specified by the user. - -**Syntax** - -``` sql -toDecimalString(number, scale) -``` - -**Parameters** - -- `number` — Value to be represented as String, [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md), [Decimal](/docs/en/sql-reference/data-types/decimal.md), -- `scale` — Number of fractional digits, [UInt8](/docs/en/sql-reference/data-types/int-uint.md). - * Maximum scale for [Decimal](/docs/en/sql-reference/data-types/decimal.md) and [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md) types is 77 (it is the maximum possible number of significant digits for Decimal), - * Maximum scale for [Float](/docs/en/sql-reference/data-types/float.md) is 60. - -**Returned value** - -- Input value represented as [String](/docs/en/sql-reference/data-types/string.md) with given number of fractional digits (scale). - The number is rounded up or down according to common arithmetic in case requested scale is smaller than original number's scale. - -**Example** - -Query: - -``` sql -SELECT toDecimalString(CAST('64.32', 'Float64'), 5); -``` - -Result: - -```response -┌toDecimalString(CAST('64.32', 'Float64'), 5)─┐ -│ 64.32000 │ -└─────────────────────────────────────────────┘ -``` - ## reinterpretAsUInt(8\|16\|32\|64) ## reinterpretAsInt(8\|16\|32\|64) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index e53104d8d71..088b1a9a1f1 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -762,44 +762,6 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; └────────────┴───────┘ ``` -## toDecimalString - -Принимает любой численный тип первым аргументом, возвращает строковое десятичное представление числа с точностью, заданной вторым аргументом. - -**Синтаксис** - -``` sql -toDecimalString(number, scale) -``` - -**Параметры** - -- `number` — Значение любого числового типа: [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md), [Float](/docs/ru/sql-reference/data-types/float.md), [Decimal](/docs/ru/sql-reference/data-types/decimal.md), -- `scale` — Требуемое количество десятичных знаков после запятой, [UInt8](/docs/ru/sql-reference/data-types/int-uint.md). - * Значение `scale` для типов [Decimal](/docs/ru/sql-reference/data-types/decimal.md) и [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md) должно не превышать 77 (так как это наибольшее количество значимых символов для этих типов), - * Значение `scale` для типа [Float](/docs/ru/sql-reference/data-types/float.md) не должно превышать 60. - -**Возвращаемое значение** - -- Строка ([String](/docs/en/sql-reference/data-types/string.md)), представляющая собой десятичное представление входного числа с заданной длиной дробной части. - При необходимости число округляется по стандартным правилам арифметики. - -**Пример использования** - -Запрос: - -``` sql -SELECT toDecimalString(CAST('64.32', 'Float64'), 5); -``` - -Результат: - -```response -┌─toDecimalString(CAST('64.32', 'Float64'), 5)┐ -│ 64.32000 │ -└─────────────────────────────────────────────┘ -``` - ## reinterpretAsUInt(8\|16\|32\|64) {#reinterpretasuint8163264} ## reinterpretAsInt(8\|16\|32\|64) {#reinterpretasint8163264} diff --git a/src/Functions/FunctionToDecimalString.cpp b/src/Functions/FunctionToDecimalString.cpp deleted file mode 100644 index fe417b19137..00000000000 --- a/src/Functions/FunctionToDecimalString.cpp +++ /dev/null @@ -1,22 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -REGISTER_FUNCTION(ToDecimalString) -{ - factory.registerFunction( - FunctionDocumentation{ - .description=R"( -Returns string representation of a number. First argument is the number of any numeric type, -second argument is the desired number of digits in fractional part. Returns String. - - )", - .examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}}, - .categories{"String"} - }, FunctionFactory::CaseInsensitive); -} - -} diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h deleted file mode 100644 index 6ae007e6b66..00000000000 --- a/src/Functions/FunctionToDecimalString.h +++ /dev/null @@ -1,312 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; - extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; -} - -class FunctionToDecimalString : public IFunction -{ -public: - static constexpr auto name = "toDecimalString"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override { return name; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isNumber(*arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal first argument for formatDecimal function: got {}, expected numeric type", - arguments[0]->getName()); - - if (!isUInt8(*arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal second argument for formatDecimal function: got {}, expected UInt8", - arguments[1]->getName()); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - -private: - /// For operations with Integer/Float - template - void vectorConstant(const FromVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, - /// and buffer resizes on-the-go. Also, .count() provided by buffer is convenient in this case. - WriteBufferFromVector buf_to(vec_to); - - for (size_t i = 0; i < input_rows_count; ++i) - { - format(vec_from[i], buf_to, precision); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - template - void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(vec_from[i], buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(value_from, buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - /// For operations with Decimal - template - void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. - constexpr size_t max_digits = std::numeric_limits::digits10; - if (precision > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - - WriteBufferFromVector buf_to(vec_to); - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - for (size_t i = 0; i < input_rows_count; ++i) - { - writeText(vec_from[i], from_scale, buf_to, true, true, precision); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(vec_from[i], from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - static void format(T value, DB::WriteBuffer & out, UInt8 precision) - { - /// Maximum of 60 is hard-coded in 'double-conversion/double-conversion.h' for floating point values, - /// Catch this here to give user a more reasonable error. - if (precision > 60) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too high precision requested for Float, must not be more than 60, got {}", Int8(precision)); - - DB::DoubleConverter::BufferType buffer; - double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; - - const auto result = DB::DoubleConverter::instance().ToFixed(value, precision, &builder); - - if (!result) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Error processing number: {}", value); - - out.write(buffer, builder.position()); - writeChar(0, out); - } - - template - static void format(T value, DB::WriteBuffer & out, UInt8 precision) - { - /// Fractional part for Integer is just trailing zeros. Let's limit it with 77 (like with Decimals). - constexpr size_t max_digits = std::numeric_limits::digits10; - if (precision > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - writeText(value, out); - if (precision > 0) [[likely]] - { - writeChar('.', out); - for (int i = 0; i < precision; ++i) - writeChar('0', out); - writeChar(0, out); - } - } - -public: - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - switch (arguments[0].type->getTypeId()) - { - case TypeIndex::UInt8: return executeType(arguments); - case TypeIndex::UInt16: return executeType(arguments); - case TypeIndex::UInt32: return executeType(arguments); - case TypeIndex::UInt64: return executeType(arguments); - case TypeIndex::UInt128: return executeType(arguments); - case TypeIndex::UInt256: return executeType(arguments); - case TypeIndex::Int8: return executeType(arguments); - case TypeIndex::Int16: return executeType(arguments); - case TypeIndex::Int32: return executeType(arguments); - case TypeIndex::Int64: return executeType(arguments); - case TypeIndex::Int128: return executeType(arguments); - case TypeIndex::Int256: return executeType(arguments); - case TypeIndex::Float32: return executeType(arguments); - case TypeIndex::Float64: return executeType(arguments); - case TypeIndex::Decimal32: return executeType(arguments); - case TypeIndex::Decimal64: return executeType(arguments); - case TypeIndex::Decimal128: return executeType(arguments); - case TypeIndex::Decimal256: return executeType(arguments); - default: - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments[0].column->getName(), getName()); - } - } - -private: - template - ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const - { - const auto * from_col_const = typeid_cast(arguments[0].column.get()); - const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); - const auto * precision_col_const = typeid_cast(arguments[1].column.get()); - - auto result_col = ColumnString::create(); - auto * result_col_string = assert_cast(result_col.get()); - ColumnString::Chars & result_chars = result_col_string->getChars(); - ColumnString::Offsets & result_offsets = result_col_string->getOffsets(); - - if constexpr (is_decimal) - { - const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); - UInt8 from_scale = from_col->getScale(); - - if (from_col) - { - if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); - else - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); - } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); - } - else - { - const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); - if (from_col) - { - if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); - else - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); - } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); - } - - return result_col; - } -}; - -} diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..0494cdf22e7 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -905,26 +905,26 @@ inline void writeText(const IPv4 & x, WriteBuffer & buf) { writeIPv4Text(x, buf) inline void writeText(const IPv6 & x, WriteBuffer & buf) { writeIPv6Text(x, buf); } template -void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, - bool fixed_fractional_length, UInt32 fractional_length) +void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) { /// If it's big integer, but the number of digits is small, /// use the implementation for smaller integers for more efficient arithmetic. + if constexpr (std::is_same_v) { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } } @@ -932,53 +932,35 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } } constexpr size_t max_digits = std::numeric_limits::digits10; assert(scale <= max_digits); - assert(fractional_length <= max_digits); - char buf[max_digits]; - memset(buf, '0', std::max(scale, fractional_length)); + memset(buf, '0', scale); T value = x; Int32 last_nonzero_pos = 0; - - if (fixed_fractional_length && fractional_length < scale) - { - T new_value = value / DecimalUtils::scaleMultiplier(scale - fractional_length - 1); - auto round_carry = new_value % 10; - value = new_value / 10; - if (round_carry >= 5) - value += 1; - } - - for (Int32 pos = fixed_fractional_length ? std::min(scale - 1, fractional_length - 1) : scale - 1; pos >= 0; --pos) + for (Int32 pos = scale - 1; pos >= 0; --pos) { auto remainder = value % 10; value /= 10; - - if (remainder != 0 && last_nonzero_pos == 0) - last_nonzero_pos = pos; - - buf[pos] += static_cast(remainder); } writeChar('.', ostr); - ostr.write(buf, fixed_fractional_length ? fractional_length : (trailing_zeros ? scale : last_nonzero_pos + 1)); + ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1); } template -void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, - bool fixed_fractional_length = false, UInt32 fractional_length = 0) +void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) { T part = DecimalUtils::getWholePart(x, scale); @@ -989,7 +971,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer writeIntText(part, ostr); - if (scale || (fixed_fractional_length && fractional_length > 0)) + if (scale) { part = DecimalUtils::getFractionalPart(x, scale); if (part || trailing_zeros) @@ -997,7 +979,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer if (part < 0) part *= T(-1); - writeDecimalFractional(part, scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(part, scale, ostr, trailing_zeros); } } } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.reference b/tests/queries/0_stateless/02676_to_decimal_string.reference deleted file mode 100644 index 4c27ee5b528..00000000000 --- a/tests/queries/0_stateless/02676_to_decimal_string.reference +++ /dev/null @@ -1,21 +0,0 @@ -2.00000000000000000000000000000000000000000000000000000000000000000000000000000 -2.12 --2.00000000000000000000000000000000000000000000000000000000000000000000000000000 --2.12 -2.987600000000000033395508580724708735942840576171875000000000 -2.15 --2.987600000000000033395508580724708735942840576171875000000000 --2.15 -64.1230010986 -64.2340000000 --64.1230010986 --64.2340000000 --32.345 -32.34500000000000000000000000000000000000000000000000000000000000000000000000000 -32.46 --64.5671232345 -128.78932312332132985464 --128.78932312332132985464 -128.78932312332132985464000000000000000000000000000000000000000000000000000000000 -128.7893231233 --128.78932312332132985464123123789323123321329854600000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql deleted file mode 100644 index 563d60c62c7..00000000000 --- a/tests/queries/0_stateless/02676_to_decimal_string.sql +++ /dev/null @@ -1,35 +0,0 @@ --- Regular types -SELECT toDecimalString(2, 77); -- more digits required than exist -SELECT toDecimalString(2.123456, 2); -- rounding -SELECT toDecimalString(-2, 77); -- more digits required than exist -SELECT toDecimalString(-2.123456, 2); -- rounding - -SELECT toDecimalString(2.9876, 60); -- more digits required than exist (took 60 as it is float by default) -SELECT toDecimalString(2.1456, 2); -- rounding -SELECT toDecimalString(-2.9876, 60); -- more digits required than exist -SELECT toDecimalString(-2.1456, 2); -- rounding - --- Float32 and Float64 tests. No sense to test big float precision -- the result will be a mess anyway. -SELECT toDecimalString(64.123::Float32, 10); -SELECT toDecimalString(64.234::Float64, 10); -SELECT toDecimalString(-64.123::Float32, 10); -SELECT toDecimalString(-64.234::Float64, 10); - --- Decimals -SELECT toDecimalString(-32.345::Decimal32(3), 3); -SELECT toDecimalString(32.345::Decimal32(3), 77); -- more digits required than exist -SELECT toDecimalString(32.456::Decimal32(3), 2); -- rounding -SELECT toDecimalString('-64.5671232345'::Decimal64(10), 10); -SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 20); -SELECT toDecimalString('-128.78932312332132985464123123'::Decimal128(26), 20); -- rounding -SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 77); -- more digits required than exist -SELECT toDecimalString('128.789323123321329854641231237893231233213298546'::Decimal256(45), 10); -- rounding -SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 77); -- more digits required than exist - --- Max number of decimal fractional digits is defined as 77 for Int/UInt/Decimal and 60 for Float. --- These values shall work OK. -SELECT toDecimalString('32.32'::Float32, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} From cda42e6dd4eaa56822ad64aad7aa09f632547d93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:05:51 +0200 Subject: [PATCH 631/871] Add a test --- .../queries/0_stateless/02831_regexp_analyze_recursion.reference | 0 tests/queries/0_stateless/02831_regexp_analyze_recursion.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02831_regexp_analyze_recursion.reference create mode 100644 tests/queries/0_stateless/02831_regexp_analyze_recursion.sql diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.reference b/tests/queries/0_stateless/02831_regexp_analyze_recursion.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql new file mode 100644 index 00000000000..018d1f031e6 --- /dev/null +++ b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql @@ -0,0 +1 @@ +SELECT match('', repeat('(', 100000)); -- { serverError 306 } From 21ffce0ff20fc7f136d8d5b05369a1abcdc01be3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:06:02 +0200 Subject: [PATCH 632/871] Check regular expression depth --- src/Common/OptimizedRegularExpression.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index c542945c78d..0b80e2f3f97 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #define MIN_LENGTH_FOR_STRSTR 3 @@ -50,6 +51,8 @@ const char * analyzeImpl( bool & is_trivial, Literals & global_alternatives) { + checkStackSize(); + /** The expression is trivial if all the metacharacters in it are escaped. * The non-alternative string is * a string outside parentheses, From de2016261ef32878456de9efae5cfab748611853 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:08:27 +0200 Subject: [PATCH 633/871] Get rid of it --- docker/test/fasttest/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 828c73e6781..e25b5fdbfed 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -141,7 +141,6 @@ function clone_submodules contrib/jemalloc contrib/replxx contrib/wyhash - contrib/hashidsxx contrib/c-ares contrib/morton-nd contrib/xxHash From 20625d75ab52319b8e67e50d2df803d0e2dc0934 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 15 Jun 2023 14:08:43 +0200 Subject: [PATCH 634/871] Fix optimize_skip_unused_shards with JOINs In case of JOIN query may contains conditions for other tables, while optimize_skip_unused_shards was pretty dumb and failed to skip such columns. Fix this by removing JOIN before applying this optimization. v2: restriction for analyzer v3: ignore 01940_custom_tld_sharding_key under analyzer Signed-off-by: Azat Khuzhin Co-Authored-By: Alexey Milovidov --- src/Storages/StorageDistributed.cpp | 42 ++++++++------ src/Storages/StorageDistributed.h | 4 +- tests/analyzer_tech_debt.txt | 2 + ...optimize_skip_unused_shards_join.reference | 0 ...02790_optimize_skip_unused_shards_join.sql | 55 +++++++++++++++++++ 5 files changed, 84 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference create mode 100644 tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0727658160c..1a99d272cab 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -75,6 +75,7 @@ #include #include #include +#include #include #include @@ -434,7 +435,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( { /// Always calculate optimized cluster here, to avoid conditions during read() /// (Anyway it will be calculated in the read()) - ClusterPtr optimized_cluster = getOptimizedCluster(local_context, storage_snapshot, query_info.query); + ClusterPtr optimized_cluster = getOptimizedCluster(local_context, storage_snapshot, query_info); if (optimized_cluster) { LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", @@ -1297,7 +1298,7 @@ ClusterPtr StorageDistributed::getCluster() const } ClusterPtr StorageDistributed::getOptimizedCluster( - ContextPtr local_context, const StorageSnapshotPtr & storage_snapshot, const ASTPtr & query_ptr) const + ContextPtr local_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const { ClusterPtr cluster = getCluster(); const Settings & settings = local_context->getSettingsRef(); @@ -1306,7 +1307,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster( if (has_sharding_key && sharding_key_is_usable) { - ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, storage_snapshot, local_context); + ClusterPtr optimized = skipUnusedShards(cluster, query_info, storage_snapshot, local_context); if (optimized) return optimized; } @@ -1355,25 +1356,34 @@ IColumn::Selector StorageDistributed::createSelector(const ClusterPtr cluster, c /// using constraints from "PREWHERE" and "WHERE" conditions, otherwise returns `nullptr` ClusterPtr StorageDistributed::skipUnusedShards( ClusterPtr cluster, - const ASTPtr & query_ptr, + const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { - const auto & select = query_ptr->as(); - + const auto & select = query_info.query->as(); if (!select.prewhere() && !select.where()) - { return nullptr; - } + + /// FIXME: support analyzer + if (!query_info.syntax_analyzer_result) + return nullptr; ASTPtr condition_ast; - if (select.prewhere() && select.where()) + /// Remove JOIN from the query since it may contain a condition for other tables. + /// But only the conditions for the left table should be analyzed for shard skipping. { - condition_ast = makeASTFunction("and", select.prewhere()->clone(), select.where()->clone()); - } - else - { - condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); + ASTPtr select_without_join_ptr = select.clone(); + ASTSelectQuery select_without_join = select_without_join_ptr->as(); + TreeRewriterResult analyzer_result_without_join = *query_info.syntax_analyzer_result; + + removeJoin(select_without_join, analyzer_result_without_join, local_context); + if (!select_without_join.prewhere() && !select_without_join.where()) + return nullptr; + + if (select_without_join.prewhere() && select_without_join.where()) + condition_ast = makeASTFunction("and", select_without_join.prewhere()->clone(), select_without_join.where()->clone()); + else + condition_ast = select_without_join.prewhere() ? select_without_join.prewhere()->clone() : select_without_join.where()->clone(); } replaceConstantExpressions(condition_ast, local_context, storage_snapshot->metadata->getColumns().getAll(), shared_from_this(), storage_snapshot); @@ -1396,11 +1406,9 @@ ClusterPtr StorageDistributed::skipUnusedShards( return nullptr; } - // Can't get definite answer if we can skip any shards + // Can't get a definite answer if we can skip any shards if (!blocks) - { return nullptr; - } std::set shards; diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f45286341cf..615d6e337b6 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -182,10 +182,10 @@ private: /// Apply the following settings: /// - optimize_skip_unused_shards /// - force_optimize_skip_unused_shards - ClusterPtr getOptimizedCluster(ContextPtr, const StorageSnapshotPtr & storage_snapshot, const ASTPtr & query_ptr) const; + ClusterPtr getOptimizedCluster(ContextPtr, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const; ClusterPtr skipUnusedShards( - ClusterPtr cluster, const ASTPtr & query_ptr, const StorageSnapshotPtr & storage_snapshot, ContextPtr context) const; + ClusterPtr cluster, const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr context) const; /// This method returns optimal query processing stage. /// diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..8ffb94e17b8 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,5 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02790_optimize_skip_unused_shards_join +01940_custom_tld_sharding_key diff --git a/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql new file mode 100644 index 00000000000..0773e0a9a5e --- /dev/null +++ b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql @@ -0,0 +1,55 @@ +-- Issue: https://github.com/ClickHouse/ClickHouse/issues/15995 + +DROP TABLE IF EXISTS outer; +DROP TABLE IF EXISTS inner; + +DROP TABLE IF EXISTS outer_distributed; +DROP TABLE IF EXISTS inner_distributed; + +CREATE TABLE IF NOT EXISTS outer +( + `id` UInt64, + `organization_id` UInt64, + `version` UInt64 +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY organization_id % 8 +ORDER BY (organization_id, id); + +CREATE TABLE inner +( + `id` UInt64, + `outer_id` UInt64, + `organization_id` UInt64, + `version` UInt64, + `date` Date +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY toYYYYMM(date) +ORDER BY (organization_id, outer_id); + +CREATE TABLE inner_distributed AS inner +ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'inner', intHash64(organization_id)); + +CREATE TABLE outer_distributed AS outer +ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'outer', intHash64(organization_id)); + +SELECT + sum(if(inner_distributed.id != 0, 1, 0)) AS total, + inner_distributed.date AS date +FROM outer_distributed AS outer_distributed +FINAL +LEFT JOIN +( + SELECT + inner_distributed.outer_id AS outer_id, + inner_distributed.id AS id, + inner_distributed.date AS date + FROM inner_distributed AS inner_distributed + FINAL + WHERE inner_distributed.organization_id = 15078 +) AS inner_distributed ON inner_distributed.outer_id = outer_distributed.id +WHERE (outer_distributed.organization_id = 15078) AND (date != toDate('1970-01-01')) +GROUP BY date +ORDER BY date DESC +SETTINGS distributed_product_mode = 'local', optimize_skip_unused_shards = 1; From c3c6560c9511163fc14eac2be8f89c30d3bcce1d Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 22 Jul 2023 08:44:19 +0000 Subject: [PATCH 635/871] Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..19b90a39800 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,4 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02815_range_dict_no_direct_join From 5ca6c97832f786e6e3be085e3ec79829f9233cdd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 22 Jul 2023 12:03:20 +0200 Subject: [PATCH 636/871] Update gtest_lru_file_cache.cpp --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 12e7d9372f7..dab14a66ed7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -489,7 +489,6 @@ TEST_F(FileCacheTest, get) download(file_segment); ASSERT_EQ(file_segment.state(), State::DOWNLOADED); - file_segment.completePartAndResetDownloader(); other_1.join(); From 8c0113e02bb19a4137871f43bbbe9b00702f8681 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jul 2023 15:54:52 +0200 Subject: [PATCH 637/871] Sanitize compose files in integration-tests-runner --- .../runner/compose/docker_compose_meili.yml | 5 ++-- .../runner/compose/docker_compose_mysql.yml | 10 +++---- .../compose/docker_compose_mysql_8_0.yml | 8 +++--- .../compose/docker_compose_mysql_cluster.yml | 26 +++++++++---------- .../compose/docker_compose_postgres.yml | 6 ++--- .../docker_compose_zookeeper_secure.yml | 6 ++--- 6 files changed, 30 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_meili.yml b/docker/test/integration/runner/compose/docker_compose_meili.yml index c734c43b4c6..c1fad4aca87 100644 --- a/docker/test/integration/runner/compose/docker_compose_meili.yml +++ b/docker/test/integration/runner/compose/docker_compose_meili.yml @@ -1,16 +1,15 @@ version: '2.3' services: meili1: - image: getmeili/meilisearch:v0.27.0 + image: getmeili/meilisearch:v0.27.0 restart: always ports: - ${MEILI_EXTERNAL_PORT:-7700}:${MEILI_INTERNAL_PORT:-7700} meili_secure: - image: getmeili/meilisearch:v0.27.0 + image: getmeili/meilisearch:v0.27.0 restart: always ports: - ${MEILI_SECURE_EXTERNAL_PORT:-7700}:${MEILI_SECURE_INTERNAL_PORT:-7700} environment: MEILI_MASTER_KEY: "password" - diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 6b98a372bd0..103fe2769e9 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -9,10 +9,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-1.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/error.log @@ -21,4 +21,4 @@ services: volumes: - type: ${MYSQL_LOGS_FS:-tmpfs} source: ${MYSQL_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index d5fb5a53aaf..9c9c7430cec 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -9,9 +9,9 @@ services: DATADIR: /mysql/ expose: - ${MYSQL8_PORT:-3306} - command: --server_id=100 --log-bin='mysql-bin-1.log' - --default_authentication_plugin='mysql_native_password' - --default-time-zone='+3:00' --gtid-mode="ON" + command: --server_id=100 --log-bin='mysql-bin-1.log' + --default_authentication_plugin='mysql_native_password' + --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/error.log @@ -20,4 +20,4 @@ services: volumes: - type: ${MYSQL8_LOGS_FS:-tmpfs} source: ${MYSQL8_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml index 8e145a3b408..73f9e39f0d6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -9,10 +9,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-2.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-2.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/2_error.log @@ -31,10 +31,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-3.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-3.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/3_error.log @@ -53,10 +53,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-4.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-4.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/4_error.log @@ -65,4 +65,4 @@ services: volumes: - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} source: ${MYSQL_CLUSTER_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 1fb6b7a1410..2ef7eb17395 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -12,9 +12,9 @@ services: timeout: 5s retries: 5 networks: - default: - aliases: - - postgre-sql.local + default: + aliases: + - postgre-sql.local environment: POSTGRES_HOST_AUTH_METHOD: "trust" POSTGRES_PASSWORD: mysecretpassword diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml index 7a1c32e0023..b5dbae423b2 100644 --- a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml +++ b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml @@ -12,7 +12,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind @@ -37,7 +37,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind @@ -61,7 +61,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind From c6e9f621f1cb19ac0ac0aa682339a3366efc89aa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jul 2023 15:36:27 +0200 Subject: [PATCH 638/871] Replace kssenii/nginx-test:1.1 by clickhouse/nginx-dav --- docker/images.json | 4 +++ docker/test/integration/nginx_dav/Dockerfile | 6 +++++ .../test/integration/nginx_dav/default.conf | 25 +++++++++++++++++++ .../runner/compose/docker_compose_nginx.yml | 2 +- .../integration/runner/dockerd-entrypoint.sh | 1 + tests/ci/integration_test_check.py | 13 +++++----- tests/integration/ci-runner.py | 1 + tests/integration/runner | 2 ++ 8 files changed, 47 insertions(+), 7 deletions(-) create mode 100644 docker/test/integration/nginx_dav/Dockerfile create mode 100644 docker/test/integration/nginx_dav/default.conf diff --git a/docker/images.json b/docker/images.json index e8fc329a640..9e4d19d30aa 100644 --- a/docker/images.json +++ b/docker/images.json @@ -161,5 +161,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/integration/nginx_dav": { + "name": "clickhouse/nginx-dav", + "dependent": [] } } diff --git a/docker/test/integration/nginx_dav/Dockerfile b/docker/test/integration/nginx_dav/Dockerfile new file mode 100644 index 00000000000..42c1244f6dc --- /dev/null +++ b/docker/test/integration/nginx_dav/Dockerfile @@ -0,0 +1,6 @@ +FROM nginx:alpine-slim + +COPY default.conf /etc/nginx/conf.d/ + +RUN mkdir /usr/share/nginx/files/ \ + && chown nginx: /usr/share/nginx/files/ -R diff --git a/docker/test/integration/nginx_dav/default.conf b/docker/test/integration/nginx_dav/default.conf new file mode 100644 index 00000000000..466d0584a2d --- /dev/null +++ b/docker/test/integration/nginx_dav/default.conf @@ -0,0 +1,25 @@ +server { + listen 80; + + #root /usr/share/nginx/test.com; + index index.html index.htm; + + server_name test.com localhost; + + location / { + expires max; + root /usr/share/nginx/files; + client_max_body_size 20m; + client_body_temp_path /usr/share/nginx/tmp; + dav_methods PUT; # Allowed methods, only PUT is necessary + + create_full_put_path on; # nginx automatically creates nested directories + dav_access user:rw group:r all:r; # access permissions for files + + limit_except GET { + allow all; + } + } + + error_page 405 =200 $uri; +} diff --git a/docker/test/integration/runner/compose/docker_compose_nginx.yml b/docker/test/integration/runner/compose/docker_compose_nginx.yml index d0fb9fc1ff4..38d2a6d84c8 100644 --- a/docker/test/integration/runner/compose/docker_compose_nginx.yml +++ b/docker/test/integration/runner/compose/docker_compose_nginx.yml @@ -5,7 +5,7 @@ services: # Files will be put into /usr/share/nginx/files. nginx: - image: kssenii/nginx-test:1.1 + image: clickhouse/nginx-dav:${DOCKER_NGINX_DAV_TAG:-latest} restart: always ports: - 80:80 diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 3c4ff522b36..66750f2089a 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -73,6 +73,7 @@ export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} +export DOCKER_NGINX_DAV_TAG=${DOCKER_NGINX_DAV_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 222b2197117..d5b70720ee9 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -37,19 +37,20 @@ from upload_result_helper import upload_results # When update, update -# integration/ci-runner.py:ClickhouseIntegrationTestsRunner.get_images_names too +# tests/integration/ci-runner.py:ClickhouseIntegrationTestsRunner.get_images_names too IMAGES = [ + "clickhouse/dotnet-client", + "clickhouse/integration-helper", + "clickhouse/integration-test", "clickhouse/integration-tests-runner", + "clickhouse/kerberized-hadoop", + "clickhouse/kerberos-kdc", "clickhouse/mysql-golang-client", "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", + "clickhouse/nginx-dav", "clickhouse/postgresql-java-client", - "clickhouse/integration-test", - "clickhouse/kerberos-kdc", - "clickhouse/kerberized-hadoop", - "clickhouse/integration-helper", - "clickhouse/dotnet-client", ] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index a6e9716dc20..8af86e37cce 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -313,6 +313,7 @@ class ClickhouseIntegrationTestsRunner: "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", + "clickhouse/nginx-dav", "clickhouse/postgresql-java-client", ] diff --git a/tests/integration/runner b/tests/integration/runner index 1b902803741..f12b62007a5 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -351,6 +351,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_KERBERIZED_HADOOP_TAG", tag) elif image == "clickhouse/kerberos-kdc": env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) + elif image == "clickhouse/nginx-dav": + env_tags += "-e {}={} ".format("DOCKER_NGINX_DAV_TAG", tag) else: logging.info("Unknown image %s" % (image)) From cf01800a63c9f067be4ca398287abc12ba0066c3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jul 2023 15:39:26 +0200 Subject: [PATCH 639/871] Sort DOCKER_*_TAG stuff --- .../integration/runner/dockerd-entrypoint.sh | 10 ++++----- tests/integration/runner | 22 +++++++++---------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 66750f2089a..b05aef76faf 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -64,16 +64,16 @@ export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge export DOCKER_BASE_TAG=${DOCKER_BASE_TAG:=latest} -export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest} -export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_DOTNET_CLIENT_TAG=${DOCKER_DOTNET_CLIENT_TAG:=latest} +export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest} +export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} +export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} +export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} -export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} -export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} -export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} export DOCKER_NGINX_DAV_TAG=${DOCKER_NGINX_DAV_TAG:=latest} +export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index f12b62007a5..4c2b1054538 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -331,18 +331,8 @@ if __name__ == "__main__": if args.docker_compose_images_tags is not None: for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") - if image == "clickhouse/mysql-golang-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) - elif image == "clickhouse/dotnet-client": + if image == "clickhouse/dotnet-client": env_tags += "-e {}={} ".format("DOCKER_DOTNET_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-java-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-js-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-php-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) - elif image == "clickhouse/postgresql-java-client": - env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "clickhouse/integration-helper": env_tags += "-e {}={} ".format("DOCKER_HELPER_TAG", tag) elif image == "clickhouse/integration-test": @@ -351,8 +341,18 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_KERBERIZED_HADOOP_TAG", tag) elif image == "clickhouse/kerberos-kdc": env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) + elif image == "clickhouse/mysql-golang-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-java-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-js-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-php-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "clickhouse/nginx-dav": env_tags += "-e {}={} ".format("DOCKER_NGINX_DAV_TAG", tag) + elif image == "clickhouse/postgresql-java-client": + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) else: logging.info("Unknown image %s" % (image)) From 363201270c00f0ebfa61e80471b372f434370380 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:49:53 +0300 Subject: [PATCH 640/871] Update 01710_query_log_with_projection_info.sql --- .../0_stateless/01710_query_log_with_projection_info.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql index 25e7e8fed60..cd84b392fe5 100644 --- a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql @@ -62,3 +62,5 @@ FROM system.query_log WHERE current_database=currentDatabase() and query = 'SELECT min(id) FROM t FORMAT Null;'; + +DROP TABLE t; From dab954a92d7893a7ebbef9cda0a3aedf63a96a50 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 22 Jul 2023 18:10:54 +0200 Subject: [PATCH 641/871] do not throw exception in OptimizedRegularExpressionImpl::analyze --- src/Common/OptimizedRegularExpression.cpp | 13 +++++++++++-- .../0_stateless/02831_regexp_analyze_recursion.sql | 2 +- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 0b80e2f3f97..918ebd75fc0 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -423,6 +423,7 @@ void OptimizedRegularExpressionImpl::analyze( bool & is_trivial, bool & required_substring_is_prefix, std::vector & alternatives) +try { Literals alternative_literals; Literal required_literal; @@ -432,12 +433,20 @@ void OptimizedRegularExpressionImpl::analyze( for (auto & lit : alternative_literals) alternatives.push_back(std::move(lit.literal)); } +catch(...) +{ + required_substring = ""; + is_trivial = false; + required_substring_is_prefix = false; + alternatives.clear(); + std::cerr << "Analyze RegularExpression failed, got error: {}" << DB::getCurrentExceptionMessage(false) << "\n"; +} template OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(const std::string & regexp_, int options) { - std::vector alternativesDummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used. - analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternativesDummy); + std::vector alternatives_dummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used. + analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternatives_dummy); /// Just three following options are supported diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql index 018d1f031e6..a2075ae903b 100644 --- a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql +++ b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql @@ -1 +1 @@ -SELECT match('', repeat('(', 100000)); -- { serverError 306 } +SELECT match('', repeat('(', 100000)); -- { serverError 427 } From c60090ccbd30143d44ab715b8b7b5e0060a2095f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 22 Jul 2023 17:43:22 +0000 Subject: [PATCH 642/871] Add test with materialize() + fix --- ...2810_fix_remove_dedundant_distinct_view.reference | 12 ++++++++++-- .../02810_fix_remove_dedundant_distinct_view.sql | 10 +++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference index 01f14f82e94..ec714a5df07 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference @@ -8,6 +8,14 @@ FROM ) WHERE explain ILIKE '%distinct%'; 2 -SELECT DISTINCT x FROM tab_v; -2 +SELECT DISTINCT x FROM tab_v ORDER BY x; 1 +2 +-- explicitly checking that materialize() doesn't affect the result, - redundant DISTINCT is still removed +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM (SELECT materialize(x) as x FROM (select DISTINCT x from tab)) +) +WHERE explain ILIKE '%distinct%'; +2 diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql index 99fc24dae8b..ca0a2edd99d 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -19,4 +19,12 @@ FROM ) WHERE explain ILIKE '%distinct%'; -SELECT DISTINCT x FROM tab_v; +SELECT DISTINCT x FROM tab_v ORDER BY x; + +-- explicitly checking that materialize() doesn't affect the result, - redundant DISTINCT is still removed +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM (SELECT materialize(x) as x FROM (select DISTINCT x from tab)) +) +WHERE explain ILIKE '%distinct%'; From afdda489bdfb27d1db2a7554223f5dfcb8cca7b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 19:53:16 +0200 Subject: [PATCH 643/871] 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 429eecbc936..270af6e5c17 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 4204279a746..07a806eddf9 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 644/871] 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 270af6e5c17..ef578b526da 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 07a806eddf9..638cc66682d 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 2c6bc318476ce98b916cd2ffb6a9a44f5a5488f8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 22 Jul 2023 18:07:22 +0000 Subject: [PATCH 645/871] Test is not dependent on new analyzer --- .../0_stateless/02810_fix_remove_dedundant_distinct_view.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql index ca0a2edd99d..10a68721c51 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -1,5 +1,3 @@ -set allow_experimental_analyzer=1; - drop table if exists tab_v; drop table if exists tab; create table tab (x UInt64, y UInt64) engine MergeTree() order by (x, y); From d25cd0d0b635196b1a4cb2178d93b7060bf02819 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 20:21:33 +0200 Subject: [PATCH 646/871] Partial revert --- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 +- .../0_stateless/02360_rename_table_along_with_log_name.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index fccac57aea8..dacb609d790 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -4,13 +4,13 @@ unset CLICKHOUSE_LOG_COMMENT CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh index c07dcdd549b..e8c7f844b5c 100755 --- a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -1,7 +1,6 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -12,6 +11,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" $CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" $CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name $CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" From 0b258dda4ee618a4d002e2b5246d68bbd2c77c7e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 21 Jul 2023 08:31:45 +0200 Subject: [PATCH 647/871] Reproducible builds for Rust From now on cargo will not download anything from the internet during builds. This step had been moved for docker image builds (via cargo vendor). And now cargo inside docker.io/clickhouse/binary-builder will not use any crates from the internet, so we don't need to add --offline for cargo commands in cmake (corrosion_import_crate()). Also the docker build command had been adjusted to allow following symlinks inside build context, by using tar, this is required for Rust packages. Note, that to make proper Cargo.lock that could be vendored I did the following: - per-project locks had been removed (since there is no automatic way to sync the workspace Cargo.lock with per-project Cargo.lock, since cargo update/generate-lockfile will use only per-project Cargo.toml files apparently, -Z minimal-versions does not helps either) - and to generate Cargo.lock with less changes I've pinned version in the Cargo.toml strictly, i.e. not 'foo = "0.1"' but 'foo = "=0.1"' then the Cargo.lock for workspace had been generated and afterwards I've reverted this part. Plus I have to update the dependencies afterwards, since otherwise there are conflicts with dependencies for std library. Non trivial. Signed-off-by: Azat Khuzhin --- .gitignore | 2 + docker/packager/binary/Dockerfile | 27 ++ docker/packager/binary/rust | 1 + rust/.dockerignore | 4 + rust/.gitignore | 4 + rust/BLAKE3/Cargo.lock | 92 ----- rust/CMakeLists.txt | 2 + rust/{skim => }/Cargo.lock | 519 +++++++++++++++++++++++++-- rust/Cargo.toml | 12 + rust/prql/Cargo.lock | 569 ------------------------------ tests/ci/docker_images_check.py | 33 +- tests/ci/docker_test.py | 12 +- 12 files changed, 582 insertions(+), 695 deletions(-) create mode 120000 docker/packager/binary/rust create mode 100644 rust/.dockerignore create mode 100644 rust/.gitignore delete mode 100644 rust/BLAKE3/Cargo.lock rename rust/{skim => }/Cargo.lock (66%) create mode 100644 rust/Cargo.toml delete mode 100644 rust/prql/Cargo.lock diff --git a/.gitignore b/.gitignore index 39d6f3f9fc8..5341f23a94f 100644 --- a/.gitignore +++ b/.gitignore @@ -69,6 +69,7 @@ cmake-build-* *.pyc __pycache__ *.pytest_cache +.mypy_cache test.cpp CPackConfig.cmake @@ -167,3 +168,4 @@ tests/integration/**/_gen /rust/**/target # It is autogenerated from *.in /rust/**/.cargo/config.toml +/rust/**/vendor diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 897bcd24d04..99e748c41d4 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -58,6 +58,33 @@ RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ rustup target add aarch64-apple-darwin && \ rustup target add powerpc64le-unknown-linux-gnu +# Create vendor cache for cargo. +# +# Note, that the config.toml for the root is used, you will not be able to +# install any other crates, except those which had been vendored (since if +# there is "replace-with" for some source, then cargo will not look to other +# remotes except this). +# +# Notes for the command itself: +# - --chown is required to preserve the rights +# - unstable-options for -C +# - chmod is required to fix the permissions, since builds are running from a different user +# - copy of the Cargo.lock is required for proper dependencies versions +# - cargo vendor --sync is requried to overcome [1] bug. +# +# [1]: https://github.com/rust-lang/wg-cargo-std-aware/issues/23 +COPY --chown=root:root /rust /rust/packages +RUN cargo -Z unstable-options -C /rust/packages vendor > $CARGO_HOME/config.toml && \ + cp "$(rustc --print=sysroot)"/lib/rustlib/src/rust/Cargo.lock "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/ && \ + cargo -Z unstable-options -C /rust/packages vendor --sync "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/Cargo.toml && \ + rm "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/Cargo.lock && \ + sed -i "s#\"vendor\"#\"/rust/vendor\"#" $CARGO_HOME/config.toml && \ + cat $CARGO_HOME/config.toml && \ + mv /rust/packages/vendor /rust/vendor && \ + chmod -R o=r+X /rust/vendor && \ + ls -R -l /rust/packages && \ + rm -r /rust/packages + # NOTE: Seems like gcc-11 is too new for ubuntu20 repository # A cross-linker for RISC-V 64 (we need it, because LLVM's LLD does not work): RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ diff --git a/docker/packager/binary/rust b/docker/packager/binary/rust new file mode 120000 index 00000000000..742dc49e9ac --- /dev/null +++ b/docker/packager/binary/rust @@ -0,0 +1 @@ +../../../rust \ No newline at end of file diff --git a/rust/.dockerignore b/rust/.dockerignore new file mode 100644 index 00000000000..6b761aa401c --- /dev/null +++ b/rust/.dockerignore @@ -0,0 +1,4 @@ +# Just in case ignore any cargo stuff (and just in case someone will run this +# docker build locally with build context using folder root): +target +vendor diff --git a/rust/.gitignore b/rust/.gitignore new file mode 100644 index 00000000000..f850cd563c9 --- /dev/null +++ b/rust/.gitignore @@ -0,0 +1,4 @@ +# This is for tar --exclude-vcs-ignores (and just in case someone will run +# docker build locally with build context created via tar): +target +vendor diff --git a/rust/BLAKE3/Cargo.lock b/rust/BLAKE3/Cargo.lock deleted file mode 100644 index 9ac60773732..00000000000 --- a/rust/BLAKE3/Cargo.lock +++ /dev/null @@ -1,92 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "_ch_rust_blake3" -version = "0.1.0" -dependencies = [ - "blake3", - "libc", -] - -[[package]] -name = "arrayref" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" - -[[package]] -name = "arrayvec" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" - -[[package]] -name = "blake3" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "526c210b4520e416420759af363083471656e819a75e831b8d2c9d5a584f2413" -dependencies = [ - "arrayref", - "arrayvec", - "cc", - "cfg-if", - "constant_time_eq", - "digest", -] - -[[package]] -name = "cc" -version = "1.0.73" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "constant_time_eq" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" - -[[package]] -name = "digest" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3dd60d1080a57a05ab032377049e0591415d2b31afd7028356dbf3cc6dcb066" -dependencies = [ - "generic-array", -] - -[[package]] -name = "generic-array" -version = "0.14.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bff49e947297f3312447abdca79f45f4738097cc82b06e72054d2223f601f1b9" -dependencies = [ - "typenum", - "version_check", -] - -[[package]] -name = "libc" -version = "0.2.132" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8371e4e5341c3a96db127eb2465ac681ced4c433e01dd0e938adbef26ba93ba5" - -[[package]] -name = "typenum" -version = "1.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 41451fe0a1e..ca0886cb300 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -55,6 +55,8 @@ function(clickhouse_import_crate) endif() endif() + # Note, here --offline is not used, since on CI vendor archive is used, and + # passing --offline here will be inconvenient for local development. corrosion_import_crate(NO_STD ${ARGN} PROFILE ${profile}) endfunction() diff --git a/rust/skim/Cargo.lock b/rust/Cargo.lock similarity index 66% rename from rust/skim/Cargo.lock rename to rust/Cargo.lock index f55ea8a84b0..07bbf8ba27e 100644 --- a/rust/skim/Cargo.lock +++ b/rust/Cargo.lock @@ -2,6 +2,22 @@ # It is not intended for manual editing. version = 3 +[[package]] +name = "_ch_rust_blake3" +version = "0.1.0" +dependencies = [ + "blake3", + "libc", +] + +[[package]] +name = "_ch_rust_prql" +version = "0.1.0" +dependencies = [ + "prql-compiler", + "serde_json", +] + [[package]] name = "_ch_rust_skim_rust" version = "0.1.0" @@ -12,6 +28,32 @@ dependencies = [ "term", ] +[[package]] +name = "addr2line" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + [[package]] name = "aho-corasick" version = "1.0.2" @@ -36,6 +78,31 @@ dependencies = [ "libc", ] +[[package]] +name = "anyhow" +version = "1.0.72" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b13c32d80ecc7ab747b80c3784bce54ee8a7a0cc4fbda9bf4cda2cf6fe90854" +dependencies = [ + "backtrace", +] + +[[package]] +name = "ariadne" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +dependencies = [ + "unicode-width", + "yansi", +] + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + [[package]] name = "arrayvec" version = "0.7.4" @@ -48,6 +115,21 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" +[[package]] +name = "backtrace" +version = "0.3.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + [[package]] name = "beef" version = "0.5.2" @@ -60,6 +142,29 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "blake3" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "199c42ab6972d92c9f8995f086273d25c42fc0f7b2a1fcefba465c1352d25ba5" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", + "digest", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + [[package]] name = "bumpalo" version = "3.13.0" @@ -93,6 +198,16 @@ dependencies = [ "winapi", ] +[[package]] +name = "chumsky" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" +dependencies = [ + "hashbrown 0.12.3", + "stacker", +] + [[package]] name = "codespan-reporting" version = "0.11.1" @@ -103,6 +218,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "constant_time_eq" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" + [[package]] name = "core-foundation-sys" version = "0.8.4" @@ -177,10 +298,41 @@ dependencies = [ ] [[package]] -name = "cxx" -version = "1.0.101" +name = "crypto-common" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5032837c1384de3708043de9d4e97bb91290faca6c16529a28aa340592a78166" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "cxx" +version = "1.0.102" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f68e12e817cb19eaab81aaec582b4052d07debd3c3c6b083b9d361db47c7dc9d" dependencies = [ "cc", "cxxbridge-flags", @@ -190,9 +342,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51368b3d0dbf356e10fcbfd455a038503a105ee556f7ee79b6bb8c53a7247456" +checksum = "e789217e4ab7cf8cc9ce82253180a9fe331f35f5d339f0ccfe0270b39433f397" dependencies = [ "cc", "codespan-reporting", @@ -200,24 +352,24 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] name = "cxxbridge-flags" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d9062157072e4aafc8e56ceaf8325ce850c5ae37578c852a0d4de2cecdded13" +checksum = "78a19f4c80fd9ab6c882286fa865e92e07688f4387370a209508014ead8751d0" [[package]] name = "cxxbridge-macro" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf01e8a540f5a4e0f284595834f81cf88572f244b768f051724537afa99a2545" +checksum = "b8fcfa71f66c8563c4fa9dd2bb68368d50267856f831ac5d85367e0805f9606c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] @@ -296,6 +448,17 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + [[package]] name = "dirs-next" version = "2.0.0" @@ -319,9 +482,27 @@ dependencies = [ [[package]] name = "either" -version = "1.8.1" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "fnv" @@ -338,6 +519,16 @@ dependencies = [ "thread_local", ] +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + [[package]] name = "getrandom" version = "0.2.10" @@ -349,6 +540,33 @@ dependencies = [ "wasi 0.11.0+wasi-snapshot-preview1", ] +[[package]] +name = "gimli" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + [[package]] name = "hermit-abi" version = "0.3.2" @@ -384,6 +602,31 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" + [[package]] name = "js-sys" version = "0.3.64" @@ -444,6 +687,21 @@ dependencies = [ "autocfg", ] +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + [[package]] name = "nix" version = "0.24.3" @@ -470,10 +728,20 @@ dependencies = [ ] [[package]] -name = "num-traits" -version = "0.2.15" +name = "nom" +version = "7.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "num-traits" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f30b0abd723be7e2ffca1272140fac1a2f084c77ec3e123c192b66af1ee9e6c2" dependencies = [ "autocfg", ] @@ -488,6 +756,15 @@ dependencies = [ "libc", ] +[[package]] +name = "object" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +dependencies = [ + "memchr", +] + [[package]] name = "once_cell" version = "1.18.0" @@ -509,6 +786,41 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "prql-compiler" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +dependencies = [ + "anyhow", + "ariadne", + "chumsky", + "csv", + "enum-as-inner", + "itertools", + "lazy_static", + "log", + "once_cell", + "regex", + "semver", + "serde", + "serde_json", + "serde_yaml", + "sqlformat", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "psm" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" +dependencies = [ + "cc", +] + [[package]] name = "quote" version = "1.0.31" @@ -589,12 +901,24 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + [[package]] name = "rustversion" version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" +[[package]] +name = "ryu" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" + [[package]] name = "scopeguard" version = "1.2.0" @@ -608,10 +932,57 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" [[package]] -name = "serde" -version = "1.0.171" +name = "semver" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" +checksum = "b0293b4b29daaf487284529cc2f5675b8e57c61f70167ba415a463651fd6a918" +dependencies = [ + "serde", +] + +[[package]] +name = "serde" +version = "1.0.174" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b88756493a5bd5e5395d53baa70b194b05764ab85b59e43e4b8f4e1192fa9b1" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.174" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e5c3a298c7f978e53536f95a63bdc4c4a64550582f31a0359a9afda6aede62e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.27", +] + +[[package]] +name = "serde_json" +version = "1.0.103" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d03b412469450d4404fe8499a268edd7f8b79fecb074b0d812ad64ca21f4031b" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a49e178e4452f45cb61d0cd8cebc1b0fafd3e41929e996cef79aa3aca91f574" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] [[package]] name = "skim" @@ -638,12 +1009,74 @@ dependencies = [ "vte", ] +[[package]] +name = "sqlformat" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" +dependencies = [ + "itertools", + "nom", + "unicode_categories", +] + +[[package]] +name = "sqlparser" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +dependencies = [ + "log", + "serde", +] + +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + [[package]] name = "strsim" version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + [[package]] name = "syn" version = "1.0.109" @@ -657,9 +1090,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.26" +version = "2.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45c3457aacde3c65315de5031ec191ce46604304d2446e803d71ade03308d970" +checksum = "b60f673f44a8255b9c8c657daf66a596d435f2da81a555b06dc644d080ba45e0" dependencies = [ "proc-macro2", "quote", @@ -688,22 +1121,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.43" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a35fc5b8971143ca348fa6df4f024d4d55264f3468c71ad1c2f365b0a4d58c42" +checksum = "611040a08a0439f8248d1990b111c95baa9c704c805fa1f62104b39655fd7f90" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.43" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" +checksum = "090198534930841fab3a5d1bb637cde49e339654e606195f8d9c76eeb081dc96" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] @@ -766,6 +1199,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "typenum" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" + [[package]] name = "unicode-ident" version = "1.0.11" @@ -778,12 +1217,30 @@ version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" +[[package]] +name = "unicode_categories" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28467d3e1d3c6586d8f25fa243f544f5800fec42d97032474e17222c2b75cfa" + [[package]] name = "utf8parse" version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + [[package]] name = "vte" version = "0.11.1" @@ -838,7 +1295,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", "wasm-bindgen-shared", ] @@ -860,7 +1317,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -967,3 +1424,9 @@ name = "windows_x86_64_msvc" version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/rust/Cargo.toml b/rust/Cargo.toml new file mode 100644 index 00000000000..2a2b582cea8 --- /dev/null +++ b/rust/Cargo.toml @@ -0,0 +1,12 @@ +# workspace is required to vendor crates for all packages. +[workspace] +members = [ + "BLAKE3", + "skim", + "prql", +] +resolver = "2" + +# FIXME: even though the profiles should be defined in the main cargo config we +# cannot do this yet, since we compile each package separatelly, so you should +# ignore warning from cargo about this. diff --git a/rust/prql/Cargo.lock b/rust/prql/Cargo.lock deleted file mode 100644 index da94e4ca852..00000000000 --- a/rust/prql/Cargo.lock +++ /dev/null @@ -1,569 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "_ch_rust_prql" -version = "0.1.0" -dependencies = [ - "prql-compiler", - "serde_json", -] - -[[package]] -name = "addr2line" -version = "0.20.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" -dependencies = [ - "gimli", -] - -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - -[[package]] -name = "ahash" -version = "0.7.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" -dependencies = [ - "getrandom", - "once_cell", - "version_check", -] - -[[package]] -name = "aho-corasick" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" -dependencies = [ - "memchr", -] - -[[package]] -name = "anyhow" -version = "1.0.71" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" -dependencies = [ - "backtrace", -] - -[[package]] -name = "ariadne" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" -dependencies = [ - "unicode-width", - "yansi", -] - -[[package]] -name = "backtrace" -version = "0.3.68" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" -dependencies = [ - "addr2line", - "cc", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", -] - -[[package]] -name = "cc" -version = "1.0.79" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "chumsky" -version = "0.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" -dependencies = [ - "hashbrown 0.12.3", - "stacker", -] - -[[package]] -name = "csv" -version = "1.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" -dependencies = [ - "csv-core", - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "csv-core" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" -dependencies = [ - "memchr", -] - -[[package]] -name = "either" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" - -[[package]] -name = "enum-as-inner" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "equivalent" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" - -[[package]] -name = "getrandom" -version = "0.2.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" -dependencies = [ - "cfg-if", - "libc", - "wasi", -] - -[[package]] -name = "gimli" -version = "0.27.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" - -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" -dependencies = [ - "ahash", -] - -[[package]] -name = "hashbrown" -version = "0.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" - -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - -[[package]] -name = "indexmap" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" -dependencies = [ - "equivalent", - "hashbrown 0.14.0", -] - -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - -[[package]] -name = "itoa" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b02a5381cc465bd3041d84623d0fa3b66738b52b8e2fc3bab8ad63ab032f4a" - -[[package]] -name = "lazy_static" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" - -[[package]] -name = "libc" -version = "0.2.147" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" - -[[package]] -name = "log" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" - -[[package]] -name = "memchr" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" - -[[package]] -name = "minimal-lexical" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" - -[[package]] -name = "miniz_oxide" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" -dependencies = [ - "adler", -] - -[[package]] -name = "nom" -version = "7.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" -dependencies = [ - "memchr", - "minimal-lexical", -] - -[[package]] -name = "object" -version = "0.31.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" -dependencies = [ - "memchr", -] - -[[package]] -name = "once_cell" -version = "1.18.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" - -[[package]] -name = "proc-macro2" -version = "1.0.63" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" -dependencies = [ - "unicode-ident", -] - -[[package]] -name = "prql-compiler" -version = "0.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" -dependencies = [ - "anyhow", - "ariadne", - "chumsky", - "csv", - "enum-as-inner", - "itertools", - "lazy_static", - "log", - "once_cell", - "regex", - "semver", - "serde", - "serde_json", - "serde_yaml", - "sqlformat", - "sqlparser", - "strum", - "strum_macros", -] - -[[package]] -name = "psm" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" -dependencies = [ - "cc", -] - -[[package]] -name = "quote" -version = "1.0.29" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" -dependencies = [ - "proc-macro2", -] - -[[package]] -name = "regex" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" -dependencies = [ - "aho-corasick", - "memchr", - "regex-automata", - "regex-syntax", -] - -[[package]] -name = "regex-automata" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" -dependencies = [ - "aho-corasick", - "memchr", - "regex-syntax", -] - -[[package]] -name = "regex-syntax" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" - -[[package]] -name = "rustc-demangle" -version = "0.1.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" - -[[package]] -name = "rustversion" -version = "1.0.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc31bd9b61a32c31f9650d18add92aa83a49ba979c143eefd27fe7177b05bd5f" - -[[package]] -name = "ryu" -version = "1.0.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe232bdf6be8c8de797b22184ee71118d63780ea42ac85b61d1baa6d3b782ae9" - -[[package]] -name = "semver" -version = "1.0.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" -dependencies = [ - "serde", -] - -[[package]] -name = "serde" -version = "1.0.166" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d01b7404f9d441d3ad40e6a636a7782c377d2abdbe4fa2440e2edcc2f4f10db8" -dependencies = [ - "serde_derive", -] - -[[package]] -name = "serde_derive" -version = "1.0.166" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dd83d6dde2b6b2d466e14d9d1acce8816dedee94f735eac6395808b3483c6d6" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.23", -] - -[[package]] -name = "serde_json" -version = "1.0.100" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" -dependencies = [ - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "serde_yaml" -version = "0.9.22" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "452e67b9c20c37fa79df53201dc03839651086ed9bbe92b3ca585ca9fdaa7d85" -dependencies = [ - "indexmap", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - -[[package]] -name = "sqlformat" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" -dependencies = [ - "itertools", - "nom", - "unicode_categories", -] - -[[package]] -name = "sqlparser" -version = "0.33.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" -dependencies = [ - "log", - "serde", -] - -[[package]] -name = "stacker" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" -dependencies = [ - "cc", - "cfg-if", - "libc", - "psm", - "winapi", -] - -[[package]] -name = "strum" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" -dependencies = [ - "strum_macros", -] - -[[package]] -name = "strum_macros" -version = "0.24.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "rustversion", - "syn 1.0.109", -] - -[[package]] -name = "syn" -version = "1.0.109" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "syn" -version = "2.0.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "unicode-ident" -version = "1.0.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73" - -[[package]] -name = "unicode-width" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" - -[[package]] -name = "unicode_categories" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" - -[[package]] -name = "unsafe-libyaml" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" - -[[package]] -name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" - -[[package]] -name = "winapi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" -dependencies = [ - "winapi-i686-pc-windows-gnu", - "winapi-x86_64-pc-windows-gnu", -] - -[[package]] -name = "winapi-i686-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" - -[[package]] -name = "winapi-x86_64-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" - -[[package]] -name = "yansi" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 16a58a90dcf..fff2975cea4 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,6 +8,7 @@ import shutil import subprocess import time import sys +from glob import glob from pathlib import Path from typing import Any, Dict, List, Optional, Set, Tuple, Union @@ -31,6 +32,17 @@ TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") ImagesDict = Dict[str, dict] +# workaround for mypy issue [1]: +# +# "Argument 1 to "map" has incompatible type overloaded function" [1] +# +# [1]: https://github.com/python/mypy/issues/9864 +# +# NOTE: simply lambda will do the trick as well, but pylint will not like it +def realpath(*args, **kwargs): + return os.path.realpath(*args, **kwargs) + + class DockerImage: def __init__( self, @@ -111,8 +123,23 @@ def get_changed_docker_images( changed_images = [] for dockerfile_dir, image_description in images_dict.items(): + source_dir = GITHUB_WORKSPACE.rstrip("/") + "/" + dockerfile_files = glob(f"{source_dir}/{dockerfile_dir}/**", recursive=True) + # resolve symlinks + dockerfile_files = list(map(realpath, dockerfile_files)) + # trim prefix to get relative path again, to match with files_changed + dockerfile_files = list(map(lambda x: x[len(source_dir) :], dockerfile_files)) + logging.info( + "Docker %s (source_dir=%s) build context for PR %s @ %s: %s", + dockerfile_dir, + source_dir, + pr_info.number, + pr_info.sha, + str(dockerfile_files), + ) + for f in files_changed: - if f.startswith(dockerfile_dir): + if f in dockerfile_files: name = image_description["name"] only_amd64 = image_description.get("only_amd64", False) logging.info( @@ -245,6 +272,8 @@ def build_and_push_one_image( cache_from = f"{cache_from} --cache-from type=registry,ref={image.repo}:{tag}" cmd = ( + # tar is requried to follow symlinks, since docker-build cannot do this + f"tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#{image.full_path.lstrip('/')}#./#' --dereference --create {image.full_path} | " "docker buildx build --builder default " f"--label build-url={GITHUB_RUN_URL} " f"{from_tag_arg}" @@ -254,7 +283,7 @@ def build_and_push_one_image( f"{cache_from} " f"--cache-to type=inline,mode=max " f"{push_arg}" - f"--progress plain {image.full_path}" + f"--progress plain -" ) logging.info("Docker command to run: %s", cmd) with TeePopen(cmd, build_log) as proc: diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..c679ab984ee 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -126,12 +126,13 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version --cache-from type=registry,ref=name:version " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --push --progress plain path", + "--cache-to type=inline,mode=max --push --progress plain -", mock_popen.call_args.args, ) self.assertTrue(result) @@ -143,12 +144,13 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version2 " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertTrue(result) @@ -160,11 +162,12 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertFalse(result) @@ -178,13 +181,14 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " "--cache-from type=registry,ref=name:cached-version " "--cache-from type=registry,ref=name:another-cached " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertFalse(result) From dc7c66396223329021641372c9156261edce5f99 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 23:44:11 +0200 Subject: [PATCH 648/871] Update comment in DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f9ed2c0d5ca..0d74e86a26d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - // hasDatabase() to avod getDatabaseName() throwing exception if database is empty. + // Callers assume that this method doesn't throw the exceptions, but getDatabaseName() can if there is no database part. auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { From 12065d94c5e35c51e3a94c1919f1a38f4723d272 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 23:59:02 +0200 Subject: [PATCH 649/871] Update comment DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0d74e86a26d..c8f332ae76d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - // Callers assume that this method doesn't throw the exceptions, but getDatabaseName() can if there is no database part. + // Callers assume that this method doesn't throw exceptions, but getDatabaseName() will throw if there is no database part. auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { From ef0dca626142322fa5420eea8fab491bb53c4ac2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 23 Jul 2023 00:37:34 +0200 Subject: [PATCH 650/871] fix style --- src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 918ebd75fc0..e636b0b987d 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -433,7 +433,7 @@ try for (auto & lit : alternative_literals) alternatives.push_back(std::move(lit.literal)); } -catch(...) +catch (...) { required_substring = ""; is_trivial = false; From 9bd8bdca98d21605f10d172b76c80951f990d965 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sun, 23 Jul 2023 01:14:26 +0200 Subject: [PATCH 651/871] Better error message in case of empty database name --- src/Interpreters/DatabaseCatalog.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c8f332ae76d..f54b0e0ab3a 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -343,9 +343,17 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { - std::lock_guard lock{databases_mutex}; // Callers assume that this method doesn't throw exceptions, but getDatabaseName() will throw if there is no database part. - auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); + // So, fail early and gracefully... + if (!table_id.hasDatabase()) + { + if (exception) + exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Empty database name")); + return {}; + } + + std::lock_guard lock{databases_mutex}; + auto it = databases.find(table_id.getDatabaseName()); if (databases.end() == it) { if (exception) From 00d6f2ee08a3e442363a078b322adab7b6988f91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 04:56:47 +0200 Subject: [PATCH 652/871] Use incbin for resources, part 1 --- .gitmodules | 3 + contrib/CMakeLists.txt | 2 +- contrib/incbin | 1 + contrib/incbin-cmake/CMakeLists.txt | 4 + contrib/nlp-data-cmake/CMakeLists.txt | 15 -- programs/install/CMakeLists.txt | 3 + programs/install/Install.cpp | 13 +- programs/keeper/CMakeLists.txt | 15 -- programs/keeper/Keeper.cpp | 6 +- programs/server/CMakeLists.txt | 12 +- programs/server/Server.cpp | 11 +- programs/server/resources.cpp | 0 src/CMakeLists.txt | 6 +- src/Common/CMakeLists.txt | 2 +- src/Common/Config/ConfigProcessor.cpp | 33 ++-- src/Common/Config/ConfigProcessor.h | 3 + src/Common/FrequencyHolder.cpp | 181 ++++++++++++++++++ src/Common/FrequencyHolder.h | 170 +--------------- src/Daemon/BaseDaemon.cpp | 1 - src/Server/WebUIRequestHandler.cpp | 14 +- src/Storages/System/CMakeLists.txt | 12 +- .../System/attachInformationSchemaTables.cpp | 24 ++- 22 files changed, 268 insertions(+), 263 deletions(-) create mode 160000 contrib/incbin create mode 100644 contrib/incbin-cmake/CMakeLists.txt delete mode 100644 contrib/nlp-data-cmake/CMakeLists.txt create mode 100644 programs/server/resources.cpp create mode 100644 src/Common/FrequencyHolder.cpp diff --git a/.gitmodules b/.gitmodules index ba71a8ae3a7..30085fb8dd4 100644 --- a/.gitmodules +++ b/.gitmodules @@ -340,3 +340,6 @@ [submodule "contrib/c-ares"] path = contrib/c-ares url = https://github.com/c-ares/c-ares.git +[submodule "contrib/incbin"] + path = contrib/incbin + url = https://github.com/graphitemaster/incbin.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 0f68c0cbc7c..fdf6e60e58f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -164,13 +164,13 @@ add_contrib (libpq-cmake libpq) add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (datasketches-cpp-cmake datasketches-cpp) +add_contrib (incbin-cmake incbin) option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES}) if (ENABLE_NLP) add_contrib (libstemmer-c-cmake libstemmer_c) add_contrib (wordnet-blast-cmake wordnet-blast) add_contrib (lemmagen-c-cmake lemmagen-c) - add_contrib (nlp-data-cmake nlp-data) add_contrib (cld2-cmake cld2) endif() diff --git a/contrib/incbin b/contrib/incbin new file mode 160000 index 00000000000..6e576cae5ab --- /dev/null +++ b/contrib/incbin @@ -0,0 +1 @@ +Subproject commit 6e576cae5ab5810f25e2631f2e0b80cbe7dc8cbf diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt new file mode 100644 index 00000000000..e64ebc99c73 --- /dev/null +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -0,0 +1,4 @@ +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/incbin") +add_library(_incbin INTERFACE) +target_include_directories(_incbin SYSTEM INTERFACE ${LIBRARY_DIR}) +add_library(ch_contrib::incbin ALIAS _incbin) diff --git a/contrib/nlp-data-cmake/CMakeLists.txt b/contrib/nlp-data-cmake/CMakeLists.txt deleted file mode 100644 index 5380269c479..00000000000 --- a/contrib/nlp-data-cmake/CMakeLists.txt +++ /dev/null @@ -1,15 +0,0 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/nlp-data") - -add_library (_nlp_data INTERFACE) - -clickhouse_embed_binaries( - TARGET nlp_dictionaries - RESOURCE_DIR "${LIBRARY_DIR}" - RESOURCES charset.zst tonality_ru.zst programming.zst -) - -add_dependencies(_nlp_data nlp_dictionaries) -target_link_libraries(_nlp_data INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") -add_library(ch_contrib::nlp_data ALIAS _nlp_data) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index c3f4d96d631..f3f562bab7c 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -10,3 +10,6 @@ set (CLICKHOUSE_INSTALL_LINK ) clickhouse_program_add_library(install) + +# For incbin +target_include_directories(clickhouse-install-lib PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/../server") diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d83e189f7ef..da2c95af62c 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -20,10 +20,7 @@ #include #include #include -#include -#include #include -#include #include #include #include @@ -35,6 +32,12 @@ #include +#include + +/// Embedded configuration files used inside the install program +INCBIN(resource_config_xml, "config.xml"); +INCBIN(resource_users_xml, "users.xml"); + /** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. * It also allows to avoid dependency on systemd, upstart, SysV init. @@ -560,7 +563,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (!fs::exists(main_config_file)) { - std::string_view main_config_content = getResource("config.xml"); + std::string_view main_config_content(reinterpret_cast(gresource_config_xmlData), gresource_config_xmlSize); if (main_config_content.empty()) { fmt::print("There is no default config.xml, you have to download it and place to {}.\n", main_config_file.string()); @@ -672,7 +675,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (!fs::exists(users_config_file)) { - std::string_view users_config_content = getResource("users.xml"); + std::string_view users_config_content(reinterpret_cast(gresource_users_xmlData), gresource_users_xmlSize); if (users_config_content.empty()) { fmt::print("There is no default users.xml, you have to download it and place to {}.\n", users_config_file.string()); diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 940e6848597..317e35959aa 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -1,16 +1,3 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - -if (OS_LINUX) - set (LINK_RESOURCE_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") - # for some reason INTERFACE linkage doesn't work for standalone binary - set (LINK_RESOURCE_LIB_STANDALONE_KEEPER "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") -endif () - -clickhouse_embed_binaries( - TARGET clickhouse_keeper_configs - RESOURCES keeper_config.xml keeper_embedded.xml -) - set(CLICKHOUSE_KEEPER_SOURCES Keeper.cpp ) @@ -29,7 +16,6 @@ set (CLICKHOUSE_KEEPER_LINK clickhouse_program_add(keeper) install(FILES keeper_config.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-keeper" COMPONENT clickhouse-keeper) -add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs) if (BUILD_STANDALONE_KEEPER) # Straight list of all required sources @@ -215,7 +201,6 @@ if (BUILD_STANDALONE_KEEPER) ${LINK_RESOURCE_LIB_STANDALONE_KEEPER} ) - add_dependencies(clickhouse-keeper clickhouse_keeper_configs) set_target_properties(clickhouse-keeper PROPERTIES RUNTIME_OUTPUT_DIRECTORY ../) if (SPLIT_DEBUG_SYMBOLS) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 6034d63a016..a38467c3369 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -457,8 +457,10 @@ try const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); std::vector extra_paths = {include_from_path}; - if (!cert_path.empty()) extra_paths.emplace_back(cert_path); - if (!key_path.empty()) extra_paths.emplace_back(key_path); + if (!cert_path.empty()) + extra_paths.emplace_back(cert_path); + if (!key_path.empty()) + extra_paths.emplace_back(key_path); /// ConfigReloader have to strict parameters which are redundant in our case auto main_config_reloader = std::make_unique( diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 855973d10e1..e008e65acf6 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -1,12 +1,8 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - set(CLICKHOUSE_SERVER_SOURCES MetricsTransmitter.cpp Server.cpp ) -set (LINK_RESOURCE_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") - set (CLICKHOUSE_SERVER_LINK PRIVATE clickhouse_aggregate_functions @@ -33,10 +29,4 @@ endif() clickhouse_program_add(server) -install(FILES config.xml users.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-server" COMPONENT clickhouse) - -clickhouse_embed_binaries( - TARGET clickhouse_server_configs - RESOURCES config.xml users.xml embedded.xml play.html dashboard.html js/uplot.js -) -add_dependencies(clickhouse-server-lib clickhouse_server_configs) +target_include_directories(clickhouse-server-lib PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 33fdcc9c1a8..229a169dc1e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -128,6 +128,10 @@ # include #endif +#include +/// A minimal file used when the server is run without installation +INCBIN(resource_embedded_xml, "embedded.xml"); + namespace CurrentMetrics { extern const Metric Revision; @@ -393,6 +397,7 @@ int Server::run() void Server::initialize(Poco::Util::Application & self) { + ConfigProcessor::registerEmbeddedConfig("config.xml", std::string_view(reinterpret_cast(gresource_embedded_xmlData), gresource_embedded_xmlSize)); BaseDaemon::initialize(self); logger().information("starting up"); @@ -1105,8 +1110,10 @@ try const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); std::vector extra_paths = {include_from_path}; - if (!cert_path.empty()) extra_paths.emplace_back(cert_path); - if (!key_path.empty()) extra_paths.emplace_back(key_path); + if (!cert_path.empty()) + extra_paths.emplace_back(cert_path); + if (!key_path.empty()) + extra_paths.emplace_back(key_path); auto main_config_reloader = std::make_unique( config_path, diff --git a/programs/server/resources.cpp b/programs/server/resources.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f870993f080..fda8bafde59 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -210,7 +210,7 @@ if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() -target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash) +target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash ch_contrib::incbin) add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) @@ -296,7 +296,7 @@ macro (dbms_target_include_directories) endforeach () endmacro () -dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") +dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src" "${ClickHouse_SOURCE_DIR}/programs/server") target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") if (TARGET ch_contrib::llvm) @@ -561,7 +561,7 @@ if (ENABLE_NLP) dbms_target_link_libraries (PUBLIC ch_contrib::stemmer) dbms_target_link_libraries (PUBLIC ch_contrib::wnb) dbms_target_link_libraries (PUBLIC ch_contrib::lemmagen) - dbms_target_link_libraries (PUBLIC ch_contrib::nlp_data) + target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_SOURCE_DIR}/contrib/nlp-data) endif() if (TARGET ch_contrib::ulid) diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index e527b3dec43..b83c8431f0a 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -9,5 +9,5 @@ if (ENABLE_EXAMPLES) endif() if (ENABLE_MYSQL) - add_subdirectory (mysqlxx) + add_subdirectory(mysqlxx) endif () diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5bbc8eae0de..c3a8f69cf3f 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -83,6 +83,13 @@ ConfigProcessor::~ConfigProcessor() Poco::Logger::destroy("ConfigProcessor"); } +static std::unordered_map embedded_configs; + +void ConfigProcessor::registerEmbeddedConfig(std::string name, std::string_view content) +{ + embedded_configs[name] = content; +} + /// Vector containing the name of the element and a sorted list of attribute names and values /// (except "remove" and "replace" attributes). @@ -281,15 +288,15 @@ void ConfigProcessor::doIncludesRecursive( { std::string value = node->nodeValue(); - bool replace_occured = false; + bool replace_occurred = false; size_t pos; while ((pos = value.find(substitution.first)) != std::string::npos) { value.replace(pos, substitution.first.length(), substitution.second); - replace_occured = true; + replace_occurred = true; } - if (replace_occured) + if (replace_occurred) node->setNodeValue(value); } } @@ -528,26 +535,14 @@ XMLDocumentPtr ConfigProcessor::processConfig( } else { - /// These embedded files added during build with some cmake magic. - /// Look at the end of programs/server/CMakeLists.txt. - std::string embedded_name; - if (path == "config.xml") - embedded_name = "embedded.xml"; - - if (path == "keeper_config.xml") - embedded_name = "keeper_embedded.xml"; - - /// When we can use config embedded in binary. - if (!embedded_name.empty()) + /// When we can use a config embedded in the binary. + if (auto it = embedded_configs.find(path); it != embedded_configs.end()) { - auto resource = getResource(embedded_name); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist and there is no embedded config", path); LOG_DEBUG(log, "There is no file '{}', will use embedded config.", path); - config = dom_parser.parseMemory(resource.data(), resource.size()); + config = dom_parser.parseMemory(it->second.data(), it->second.size()); } else - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist and there is no embedded config", path); } std::vector contributing_files; diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0ca3e46db88..eefe65ef06c 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -65,6 +65,9 @@ public: zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr, const zkutil::EventPtr & zk_changed_event = nullptr); + /// These configurations will be used if there is no configuration file. + static void registerEmbeddedConfig(std::string name, std::string_view content); + /// loadConfig* functions apply processConfig and create Poco::Util::XMLConfiguration. /// The resulting XML document is saved into a file with the name diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp new file mode 100644 index 00000000000..3b755cacacb --- /dev/null +++ b/src/Common/FrequencyHolder.cpp @@ -0,0 +1,181 @@ +#include + +#include + +/// Embedded SQL definitions +INCBIN(resource_charset_zst, "charset.zst"); +INCBIN(resource_tonality_ru_zst, "tonality_ru.zst"); +INCBIN(resource_programming_zst, "programming.zst"); + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; +} + + +FrequencyHolder & FrequencyHolder::getInstance() +{ + static FrequencyHolder instance; + return instance; +} + +FrequencyHolder::FrequencyHolder() +{ + loadEmotionalDict(); + loadEncodingsFrequency(); + loadProgrammingFrequency(); +} + +void FrequencyHolder::loadEncodingsFrequency() +{ + Poco::Logger * log = &Poco::Logger::get("EncodingsFrequency"); + + LOG_TRACE(log, "Loading embedded charset frequencies"); + + std::string_view resource(reinterpret_cast(gresource_charset_zstData), gresource_charset_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded charset frequencies"); + + String line; + UInt16 bigram; + Float64 frequency; + String charset_name; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + // Start loading a new charset + if (line.starts_with("// ")) + { + // Skip "// " + buf_line.ignore(3); + readString(charset_name, buf_line); + + /* In our dictionary we have lines with form: _ + * If we need to find language of data, we return + * If we need to find charset of data, we return . + */ + size_t sep = charset_name.find('_'); + + Encoding enc; + enc.lang = charset_name.substr(0, sep); + enc.name = charset_name.substr(sep + 1); + encodings_freq.push_back(std::move(enc)); + } + else + { + readIntText(bigram, buf_line); + buf_line.ignore(); + readFloatText(frequency, buf_line); + + encodings_freq.back().map[bigram] = frequency; + } + } + LOG_TRACE(log, "Charset frequencies was added, charsets count: {}", encodings_freq.size()); +} + +void FrequencyHolder::loadEmotionalDict() +{ + Poco::Logger * log = &Poco::Logger::get("EmotionalDict"); + LOG_TRACE(log, "Loading embedded emotional dictionary"); + + std::string_view resource(reinterpret_cast(gresource_tonality_ru_zstData), gresource_tonality_ru_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded emotional dictionary"); + + String line; + String word; + Float64 tonality; + size_t count = 0; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + readStringUntilWhitespace(word, buf_line); + buf_line.ignore(); + readFloatText(tonality, buf_line); + + StringRef ref{string_pool.insert(word.data(), word.size()), word.size()}; + emotional_dict[ref] = tonality; + ++count; + } + LOG_TRACE(log, "Emotional dictionary was added. Word count: {}", std::to_string(count)); +} + +void FrequencyHolder::loadProgrammingFrequency() +{ + Poco::Logger * log = &Poco::Logger::get("ProgrammingFrequency"); + + LOG_TRACE(log, "Loading embedded programming languages frequencies loading"); + + std::string_view resource(reinterpret_cast(gresource_programming_zstData), gresource_programming_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded programming languages frequencies"); + + String line; + String bigram; + Float64 frequency; + String programming_language; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + // Start loading a new language + if (line.starts_with("// ")) + { + // Skip "// " + buf_line.ignore(3); + readString(programming_language, buf_line); + + Language lang; + lang.name = programming_language; + programming_freq.push_back(std::move(lang)); + } + else + { + readStringUntilWhitespace(bigram, buf_line); + buf_line.ignore(); + readFloatText(frequency, buf_line); + + StringRef ref{string_pool.insert(bigram.data(), bigram.size()), bigram.size()}; + programming_freq.back().map[ref] = frequency; + } + } + LOG_TRACE(log, "Programming languages frequencies was added"); +} + +} diff --git a/src/Common/FrequencyHolder.h b/src/Common/FrequencyHolder.h index 74098598441..270e4dbbd2a 100644 --- a/src/Common/FrequencyHolder.h +++ b/src/Common/FrequencyHolder.h @@ -7,7 +7,6 @@ #include #include -#include #include #include #include @@ -20,11 +19,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int FILE_DOESNT_EXIST; -} - /// FrequencyHolder class is responsible for storing and loading dictionaries /// needed for text classification functions: /// @@ -56,11 +50,7 @@ public: using EncodingMap = HashMap; using EncodingContainer = std::vector; - static FrequencyHolder & getInstance() - { - static FrequencyHolder instance; - return instance; - } + static FrequencyHolder & getInstance(); const Map & getEmotionalDict() const { @@ -78,161 +68,11 @@ public: } private: + FrequencyHolder(); - FrequencyHolder() - { - loadEmotionalDict(); - loadEncodingsFrequency(); - loadProgrammingFrequency(); - } - - void loadEncodingsFrequency() - { - Poco::Logger * log = &Poco::Logger::get("EncodingsFrequency"); - - LOG_TRACE(log, "Loading embedded charset frequencies"); - - auto resource = getResource("charset.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded charset frequencies"); - - String line; - UInt16 bigram; - Float64 frequency; - String charset_name; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - // Start loading a new charset - if (line.starts_with("// ")) - { - // Skip "// " - buf_line.ignore(3); - readString(charset_name, buf_line); - - /* In our dictionary we have lines with form: _ - * If we need to find language of data, we return - * If we need to find charset of data, we return . - */ - size_t sep = charset_name.find('_'); - - Encoding enc; - enc.lang = charset_name.substr(0, sep); - enc.name = charset_name.substr(sep + 1); - encodings_freq.push_back(std::move(enc)); - } - else - { - readIntText(bigram, buf_line); - buf_line.ignore(); - readFloatText(frequency, buf_line); - - encodings_freq.back().map[bigram] = frequency; - } - } - LOG_TRACE(log, "Charset frequencies was added, charsets count: {}", encodings_freq.size()); - } - - void loadEmotionalDict() - { - Poco::Logger * log = &Poco::Logger::get("EmotionalDict"); - LOG_TRACE(log, "Loading embedded emotional dictionary"); - - auto resource = getResource("tonality_ru.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded emotional dictionary"); - - String line; - String word; - Float64 tonality; - size_t count = 0; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - readStringUntilWhitespace(word, buf_line); - buf_line.ignore(); - readFloatText(tonality, buf_line); - - StringRef ref{string_pool.insert(word.data(), word.size()), word.size()}; - emotional_dict[ref] = tonality; - ++count; - } - LOG_TRACE(log, "Emotional dictionary was added. Word count: {}", std::to_string(count)); - } - - void loadProgrammingFrequency() - { - Poco::Logger * log = &Poco::Logger::get("ProgrammingFrequency"); - - LOG_TRACE(log, "Loading embedded programming languages frequencies loading"); - - auto resource = getResource("programming.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded programming languages frequencies"); - - String line; - String bigram; - Float64 frequency; - String programming_language; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - // Start loading a new language - if (line.starts_with("// ")) - { - // Skip "// " - buf_line.ignore(3); - readString(programming_language, buf_line); - - Language lang; - lang.name = programming_language; - programming_freq.push_back(std::move(lang)); - } - else - { - readStringUntilWhitespace(bigram, buf_line); - buf_line.ignore(); - readFloatText(frequency, buf_line); - - StringRef ref{string_pool.insert(bigram.data(), bigram.size()), bigram.size()}; - programming_freq.back().map[ref] = frequency; - } - } - LOG_TRACE(log, "Programming languages frequencies was added"); - } + void loadEncodingsFrequency(); + void loadEmotionalDict(); + void loadProgrammingFrequency(); Arena string_pool; diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 3852ec5ada5..f61ca054b2a 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -38,7 +38,6 @@ #include #include -#include #include #include #include diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index 3997e0f19b6..cb9e8935d8c 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -6,10 +6,16 @@ #include #include -#include #include +#include + +/// Embedded HTML pages +INCBIN(resource_play_html, "play.html"); +INCBIN(resource_dashboard_html, "dashboard.html"); +INCBIN(resource_uplot_js, "js/uplot.js"); + namespace DB { @@ -34,13 +40,13 @@ void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerR if (request.getURI().starts_with("/play")) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - *response.send() << getResource("play.html"); + *response.send() << std::string_view(reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize); } else if (request.getURI().starts_with("/dashboard")) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - std::string html(getResource("dashboard.html")); + std::string html(reinterpret_cast(gresource_dashboard_htmlData), gresource_dashboard_htmlSize); /// Replace a link to external JavaScript file to embedded file. /// This allows to open the HTML without running a server and to host it on server. @@ -55,7 +61,7 @@ void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerR else if (request.getURI() == "/js/uplot.js") { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - *response.send() << getResource("js/uplot.js"); + *response.send() << std::string_view(reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize); } else { diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 1d2a3de5101..6b7d1739e33 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -43,18 +43,9 @@ list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC}) # Overlength strings set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w) -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) -clickhouse_embed_binaries( - TARGET information_schema_metadata - RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}/InformationSchema/" - RESOURCES schemata.sql tables.sql views.sql columns.sql -) - list (SORT storages_system_sources) # Reproducible build add_library(clickhouse_storages_system ${storages_system_sources}) -add_dependencies(clickhouse_storages_system information_schema_metadata) - target_link_libraries(clickhouse_storages_system PRIVATE dbms common @@ -62,5 +53,6 @@ target_link_libraries(clickhouse_storages_system PRIVATE clickhouse_common_zookeeper clickhouse_parsers Poco::JSON - INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}" ) + +target_include_directories(clickhouse_storages_system PRIVATE InformationSchema) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 61a91685324..bfc5c8c64e2 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -3,14 +3,21 @@ #include #include #include -#include +#include + +/// Embedded SQL definitions +INCBIN(resource_schemata_sql, "schemata.sql"); +INCBIN(resource_tables_sql, "tables.sql"); +INCBIN(resource_views_sql, "views.sql"); +INCBIN(resource_columns_sql, "columns.sql"); + namespace DB { /// View structures are taken from http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt -static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name) +static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name, std::string_view query) { try { @@ -21,12 +28,11 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE; String metadata_resource_name = view_name + ".sql"; - auto attach_query = getResource(metadata_resource_name); - if (attach_query.empty()) + if (query.empty()) return; ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, attach_query.data(), attach_query.data() + attach_query.size(), + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "Attach query from embedded resource " + metadata_resource_name, DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH); @@ -50,10 +56,10 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database) { - createInformationSchemaView(context, information_schema_database, "schemata"); - createInformationSchemaView(context, information_schema_database, "tables"); - createInformationSchemaView(context, information_schema_database, "views"); - createInformationSchemaView(context, information_schema_database, "columns"); + createInformationSchemaView(context, information_schema_database, "schemata", std::string_view(reinterpret_cast(gresource_schemata_sqlData), gresource_schemata_sqlSize)); + createInformationSchemaView(context, information_schema_database, "tables", std::string_view(reinterpret_cast(gresource_tables_sqlData), gresource_tables_sqlSize)); + createInformationSchemaView(context, information_schema_database, "views", std::string_view(reinterpret_cast(gresource_views_sqlData), gresource_views_sqlSize)); + createInformationSchemaView(context, information_schema_database, "columns", std::string_view(reinterpret_cast(gresource_columns_sqlData), gresource_columns_sqlSize)); } } From 4170d1458bdbccafe2f8cb2c671ee044b3efe9ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 05:25:14 +0200 Subject: [PATCH 653/871] Use incbin for resources, part 2 --- cmake/embed_binary.cmake | 58 ------------------ contrib/cctz-cmake/CMakeLists.txt | 45 ++++++++------ src/Common/Config/ConfigProcessor.cpp | 1 - src/Common/DateLUTImpl.cpp | 17 ++++-- src/Common/SymbolIndex.cpp | 61 +++---------------- src/Common/SymbolIndex.h | 30 +-------- src/Common/getResource.cpp | 52 ---------------- src/Common/getResource.h | 7 --- src/Common/tests/gtest_DateLUTImpl.cpp | 14 ++--- .../System/StorageSystemTimeZones.cpp | 7 ++- 10 files changed, 58 insertions(+), 234 deletions(-) delete mode 100644 cmake/embed_binary.cmake delete mode 100644 src/Common/getResource.cpp delete mode 100644 src/Common/getResource.h diff --git a/cmake/embed_binary.cmake b/cmake/embed_binary.cmake deleted file mode 100644 index e5428c24939..00000000000 --- a/cmake/embed_binary.cmake +++ /dev/null @@ -1,58 +0,0 @@ -# Embed a set of resource files into a resulting object file. -# -# Signature: `clickhouse_embed_binaries(TARGET RESOURCE_DIR RESOURCES ...) -# -# This will generate a static library target named ``, which contains the contents of -# each `` file. The files should be located in ``. defaults to -# ${CMAKE_CURRENT_SOURCE_DIR}, and the resources may not be empty. -# -# Each resource will result in three symbols in the final archive, based on the name ``. -# These are: -# 1. `_binary__start`: Points to the start of the binary data from ``. -# 2. `_binary__end`: Points to the end of the binary data from ``. -# 2. `_binary__size`: Points to the size of the binary data from ``. -# -# `` is a normalized name derived from ``, by replacing the characters "./-" with -# the character "_", and the character "+" with "_PLUS_". This scheme is similar to those generated -# by `ld -r -b binary`, and matches the expectations in `./base/common/getResource.cpp`. -macro(clickhouse_embed_binaries) - set(one_value_args TARGET RESOURCE_DIR) - set(resources RESOURCES) - cmake_parse_arguments(EMBED "" "${one_value_args}" ${resources} ${ARGN}) - - if (NOT DEFINED EMBED_TARGET) - message(FATAL_ERROR "A target name must be provided for embedding binary resources into") - endif() - - if (NOT DEFINED EMBED_RESOURCE_DIR) - set(EMBED_RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}") - endif() - - list(LENGTH EMBED_RESOURCES N_RESOURCES) - if (N_RESOURCES LESS 1) - message(FATAL_ERROR "The list of binary resources to embed may not be empty") - endif() - - add_library("${EMBED_TARGET}" STATIC) - set_target_properties("${EMBED_TARGET}" PROPERTIES LINKER_LANGUAGE C) - - set(EMBED_TEMPLATE_FILE "${PROJECT_SOURCE_DIR}/programs/embed_binary.S.in") - - foreach(RESOURCE_FILE ${EMBED_RESOURCES}) - set(ASSEMBLY_FILE_NAME "${RESOURCE_FILE}.S") - set(BINARY_FILE_NAME "${RESOURCE_FILE}") - - # Normalize the name of the resource. - string(REGEX REPLACE "[\./-]" "_" SYMBOL_NAME "${RESOURCE_FILE}") # - must be last in regex - string(REPLACE "+" "_PLUS_" SYMBOL_NAME "${SYMBOL_NAME}") - - # Generate the configured assembly file in the output directory. - configure_file("${EMBED_TEMPLATE_FILE}" "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}" @ONLY) - - # Set the include directory for relative paths specified for `.incbin` directive. - set_property(SOURCE "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}" APPEND PROPERTY INCLUDE_DIRECTORIES "${EMBED_RESOURCE_DIR}") - - target_sources("${EMBED_TARGET}" PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}") - set_target_properties("${EMBED_TARGET}" PROPERTIES OBJECT_DEPENDS "${RESOURCE_FILE}") - endforeach() -endmacro() diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 10070fbd949..8aa3c7886db 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -1,4 +1,3 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cctz") set (SRCS @@ -23,12 +22,10 @@ if (OS_FREEBSD) endif () # Related to time_zones table: -# StorageSystemTimeZones.generated.cpp is autogenerated each time during a build -# data in this file will be used to populate the system.time_zones table, this is specific to OS_LINUX -# as the library that's built using embedded tzdata is also specific to OS_LINUX -set(SYSTEM_STORAGE_TZ_FILE "${PROJECT_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") +# TimeZones.generated.cpp is autogenerated each time during a build +set(TIMEZONES_FILE "${CMAKE_CURRENT_BINARY_DIR}/TimeZones.generated.cpp") # remove existing copies so that its generated fresh on each build. -file(REMOVE ${SYSTEM_STORAGE_TZ_FILE}) +file(REMOVE ${TIMEZONES_FILE}) # get the list of timezones from tzdata shipped with cctz set(TZDIR "${LIBRARY_DIR}/testdata/zoneinfo") @@ -36,28 +33,36 @@ file(STRINGS "${LIBRARY_DIR}/testdata/version" TZDATA_VERSION) set_property(GLOBAL PROPERTY TZDATA_VERSION_PROP "${TZDATA_VERSION}") message(STATUS "Packaging with tzdata version: ${TZDATA_VERSION}") -set(TIMEZONE_RESOURCE_FILES) - # each file in that dir (except of tab and localtime) store the info about timezone execute_process(COMMAND bash -c "cd ${TZDIR} && find * -type f -and ! -name '*.tab' -and ! -name 'localtime' | LC_ALL=C sort | paste -sd ';' -" OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE TIMEZONES) -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} "const char * auto_time_zones[] {\n" ) +file(APPEND ${TIMEZONES_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") +file(APPEND ${TIMEZONES_FILE} "#include \n") +set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${SYSTEM_STORAGE_TZ_FILE} " \"${TIMEZONE}\",\n") - list(APPEND TIMEZONE_RESOURCE_FILES "${TIMEZONE}") + file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TIMEZONE}\");\n") + MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} " nullptr};\n") -clickhouse_embed_binaries( - TARGET tzdata - RESOURCE_DIR "${TZDIR}" - RESOURCES ${TIMEZONE_RESOURCE_FILES} -) -add_dependencies(_cctz tzdata) -target_link_libraries(_cctz INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") + +file(APPEND ${TIMEZONES_FILE} "#include \n") +file(APPEND ${TIMEZONES_FILE} "struct TimeZone { const char * name; const unsigned char * data; size_t size; };\n") +file(APPEND ${TIMEZONES_FILE} "TimeZone auto_time_zones[] {\n" ) + +set (COUNTER 1) +foreach(TIMEZONE ${TIMEZONES}) + file(APPEND ${TIMEZONES_FILE} " {\"${TIMEZONE}\", gresource_timezone${COUNTER}Data, gresource_timezone${COUNTER}Size},\n") + MATH(EXPR COUNTER "${COUNTER}+1") +endforeach(TIMEZONE) + +file(APPEND ${TIMEZONES_FILE} " {nullptr, nullptr, 0}};\n") + +add_library (tzdata ${TIMEZONES_FILE}) +target_link_libraries(tzdata ch_contrib::incbin) +target_include_directories(tzdata PRIVATE ${TZDIR}) +target_link_libraries(_cctz tzdata) add_library(ch_contrib::cctz ALIAS _cctz) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index c3a8f69cf3f..bda181eceeb 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 8146b35cc5f..3619462e79b 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -13,6 +12,10 @@ #include +/// Embedded timezones. +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; + namespace { @@ -249,9 +252,15 @@ namespace cctz_extension const std::string & name, const std::function(const std::string & name)> & fallback) { - std::string_view resource = getResource(name); - if (!resource.empty()) - return std::make_unique(resource.data(), resource.size()); + const TimeZone * timezone = auto_time_zones; + while (timezone->name != nullptr) + { + if (timezone->name == name) + break; + ++timezone; + } + if (timezone->size) + return std::make_unique(reinterpret_cast(timezone->data), timezone->size); return fallback(name); } diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index cb02bb3ff75..ac406538033 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -87,50 +87,13 @@ namespace /// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object -void updateResources(ElfW(Addr) base_address, std::string_view object_name, std::string_view name, const void * address, SymbolIndex::Resources & resources) -{ - const char * char_address = static_cast(address); - - if (name.starts_with("_binary_") || name.starts_with("binary_")) - { - if (name.ends_with("_start")) - { - name = name.substr((name[0] == '_') + strlen("binary_")); - name = name.substr(0, name.size() - strlen("_start")); - - auto & resource = resources[name]; - if (!resource.base_address || resource.base_address == base_address) - { - resource.base_address = base_address; - resource.start = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) - resource.object_name = object_name; - } - } - if (name.ends_with("_end")) - { - name = name.substr((name[0] == '_') + strlen("binary_")); - name = name.substr(0, name.size() - strlen("_end")); - - auto & resource = resources[name]; - if (!resource.base_address || resource.base_address == base_address) - { - resource.base_address = base_address; - resource.end = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) - resource.object_name = object_name; - } - } - } -} - - /// Based on the code of musl-libc and the answer of Kanalpiroge on /// https://stackoverflow.com/questions/15779185/list-all-the-functions-symbols-on-the-fly-in-c-code-on-a-linux-architecture /// It does not extract all the symbols (but only public - exported and used for dynamic linking), /// but will work if we cannot find or parse ELF files. void collectSymbolsFromProgramHeaders( dl_phdr_info * info, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { /* Iterate over all headers of the current shared lib * (first call is for the executable itself) @@ -248,9 +211,6 @@ void collectSymbolsFromProgramHeaders( /// We are not interested in empty symbols. if (elf_sym[sym_index].st_size) symbols.push_back(symbol); - - /// But resources can be represented by a pair of empty symbols (indicating their boundaries). - updateResources(base_address, info->dlpi_name, symbol.name, symbol.address_begin, resources); } break; @@ -281,8 +241,7 @@ void collectSymbolsFromELFSymbolTable( const Elf & elf, const Elf::Section & symbol_table, const Elf::Section & string_table, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { /// Iterate symbol table. const ElfSym * symbol_table_entry = reinterpret_cast(symbol_table.begin()); @@ -312,8 +271,6 @@ void collectSymbolsFromELFSymbolTable( if (symbol_table_entry->st_size) symbols.push_back(symbol); - - updateResources(info->dlpi_addr, info->dlpi_name, symbol.name, symbol.address_begin, resources); } } @@ -323,8 +280,7 @@ bool searchAndCollectSymbolsFromELFSymbolTable( const Elf & elf, unsigned section_header_type, const char * string_table_name, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { std::optional symbol_table; std::optional string_table; @@ -342,7 +298,7 @@ bool searchAndCollectSymbolsFromELFSymbolTable( return false; } - collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols, resources); + collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols); return true; } @@ -351,7 +307,6 @@ void collectSymbolsFromELF( dl_phdr_info * info, std::vector & symbols, std::vector & objects, - SymbolIndex::Resources & resources, String & build_id) { String object_name; @@ -462,11 +417,11 @@ void collectSymbolsFromELF( object.name = object_name; objects.push_back(std::move(object)); - searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols, resources); + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols); /// Unneeded if they were parsed from "program headers" of loaded objects. #if defined USE_MUSL - searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols, resources); + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols); #endif } @@ -479,8 +434,8 @@ int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr) { SymbolIndex::Data & data = *reinterpret_cast(data_ptr); - collectSymbolsFromProgramHeaders(info, data.symbols, data.resources); - collectSymbolsFromELF(info, data.symbols, data.objects, data.resources, data.build_id); + collectSymbolsFromProgramHeaders(info, data.symbols); + collectSymbolsFromELF(info, data.symbols, data.objects, data.build_id); /* Continue iterations */ return 0; diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index 4fd108434d5..8c7b8971805 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -45,44 +46,15 @@ public: const std::vector & symbols() const { return data.symbols; } const std::vector & objects() const { return data.objects; } - std::string_view getResource(String name) const - { - if (auto it = data.resources.find(name); it != data.resources.end()) - return it->second.data(); - return {}; - } - /// The BuildID that is generated by compiler. String getBuildID() const { return data.build_id; } String getBuildIDHex() const; - struct ResourcesBlob - { - /// Symbol can be presented in multiple shared objects, - /// base_address will be used to compare only symbols from the same SO. - ElfW(Addr) base_address = 0; - /// Just a human name of the SO. - std::string_view object_name; - /// Data blob. - std::string_view start; - std::string_view end; - - std::string_view data() const - { - assert(end.data() >= start.data()); - return std::string_view{start.data(), static_cast(end.data() - start.data())}; - } - }; - using Resources = std::unordered_map; - struct Data { std::vector symbols; std::vector objects; String build_id; - - /// Resources (embedded binary data) are located by symbols in form of _binary_name_start and _binary_name_end. - Resources resources; }; private: Data data; diff --git a/src/Common/getResource.cpp b/src/Common/getResource.cpp deleted file mode 100644 index 72ba24c2f44..00000000000 --- a/src/Common/getResource.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include "getResource.h" -#include -#include -#include -#include - - -std::string_view getResource(std::string_view name) -{ - // Convert the resource file name into the form generated by `ld -r -b binary`. - std::string name_replaced(name); - std::replace(name_replaced.begin(), name_replaced.end(), '/', '_'); - std::replace(name_replaced.begin(), name_replaced.end(), '-', '_'); - std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); - boost::replace_all(name_replaced, "+", "_PLUS_"); - -#if defined USE_MUSL - /// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself. - return DB::SymbolIndex::instance().getResource(name_replaced); - -#else - // In most `dlsym(3)` APIs, one passes the symbol name as it appears via - // something like `nm` or `objdump -t`. For example, a symbol `_foo` would be - // looked up with the string `"_foo"`. - // - // Apple's linker is confusingly different. The NOTES on the man page for - // `dlsym(3)` claim that one looks up the symbol with "the name used in C - // source code". In this example, that would mean using the string `"foo"`. - // This apparently applies even in the case where the symbol did not originate - // from C source, such as the embedded binary resource files used here. So - // the symbol name must not have a leading `_` on Apple platforms. It's not - // clear how this applies to other symbols, such as those which _have_ a leading - // underscore in them by design, many leading underscores, etc. -#if defined OS_DARWIN - std::string prefix = "binary_"; -#else - std::string prefix = "_binary_"; -#endif - std::string symbol_name_start = prefix + name_replaced + "_start"; - std::string symbol_name_end = prefix + name_replaced + "_end"; - - const char * sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); - const char * sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); - - if (sym_start && sym_end) - { - auto resource_size = static_cast(std::distance(sym_start, sym_end)); - return { sym_start, resource_size }; - } - return {}; -#endif -} diff --git a/src/Common/getResource.h b/src/Common/getResource.h deleted file mode 100644 index 8975cc7841e..00000000000 --- a/src/Common/getResource.h +++ /dev/null @@ -1,7 +0,0 @@ -#pragma once - -#include - -/// Get resource from binary if exists. Otherwise return empty string view. -/// Resources are data that is embedded into executable at link time. -std::string_view getResource(std::string_view name); diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 04f63403ec2..b09319c78d6 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -15,7 +15,8 @@ #endif // All timezones present at build time and embedded into ClickHouse binary. -extern const char * auto_time_zones[]; +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; namespace { @@ -32,14 +33,14 @@ std::vector allTimezones(bool with_weird_offsets = true) { std::vector result; - const auto * timezone_name = auto_time_zones; - while (*timezone_name) + const TimeZone * timezone = auto_time_zones; + while (timezone->name) { - bool weird_offsets = (std::string_view(*timezone_name) == "Africa/Monrovia"); + bool weird_offsets = (std::string_view(timezone->name) == "Africa/Monrovia"); if (!weird_offsets || with_weird_offsets) - result.push_back(*timezone_name); - ++timezone_name; + result.push_back(timezone->name); + ++timezone; } return result; @@ -548,4 +549,3 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970, // {0, 0 + 11 * 3600 * 24 + 12, 11}, })) ); - diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index dc3711812a6..41227ab7780 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -4,7 +4,8 @@ #include -extern const char * auto_time_zones[]; +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; namespace DB { @@ -17,7 +18,7 @@ NamesAndTypesList StorageSystemTimeZones::getNamesAndTypes() void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - for (auto * it = auto_time_zones; *it; ++it) - res_columns[0]->insert(String(*it)); + for (auto * it = auto_time_zones; it->name != nullptr; ++it) + res_columns[0]->insert(String(it->name)); } } From c8f8a23c71dc88ab53318be369ca17b528047b05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 06:09:15 +0200 Subject: [PATCH 654/871] Fix errors --- contrib/cctz-cmake/CMakeLists.txt | 21 +++++++++++++------ src/Common/DateLUTImpl.cpp | 18 +++++++--------- src/Common/tests/gtest_DateLUTImpl.cpp | 13 ++++++------ src/Storages/System/CMakeLists.txt | 2 -- .../System/StorageSystemTimeZones.cpp | 7 +++---- 5 files changed, 31 insertions(+), 30 deletions(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 8aa3c7886db..7edeada6e59 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -48,17 +48,26 @@ foreach(TIMEZONE ${TIMEZONES}) MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} "#include \n") -file(APPEND ${TIMEZONES_FILE} "struct TimeZone { const char * name; const unsigned char * data; size_t size; };\n") -file(APPEND ${TIMEZONES_FILE} "TimeZone auto_time_zones[] {\n" ) +file(APPEND ${TIMEZONES_FILE} "const char * auto_time_zones[] {\n" ) -set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${TIMEZONES_FILE} " {\"${TIMEZONE}\", gresource_timezone${COUNTER}Data, gresource_timezone${COUNTER}Size},\n") + file(APPEND ${TIMEZONES_FILE} " \"${TIMEZONE}\",\n") MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} " {nullptr, nullptr, 0}};\n") +file(APPEND ${TIMEZONES_FILE} "};\n\n") + +file(APPEND ${TIMEZONES_FILE} "#include \n\n") +file(APPEND ${TIMEZONES_FILE} "std::string_view getTimeZone(const char * name)\n{\n" ) + +set (COUNTER 1) +foreach(TIMEZONE ${TIMEZONES}) + file(APPEND ${TIMEZONES_FILE} " if (std::string_view(\"${TIMEZONE}\") == name) return { reinterpret_cast(gresource_timezone${COUNTER}Data), gresource_timezone${COUNTER}Size };\n") + MATH(EXPR COUNTER "${COUNTER}+1") +endforeach(TIMEZONE) + +file(APPEND ${TIMEZONES_FILE} " return {};\n") +file(APPEND ${TIMEZONES_FILE} "}\n") add_library (tzdata ${TIMEZONES_FILE}) target_link_libraries(tzdata ch_contrib::incbin) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 3619462e79b..d5e04238ef9 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,11 +10,12 @@ #include #include #include +#include /// Embedded timezones. -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +std::string_view getTimeZone(const char * name); + namespace { @@ -252,15 +253,10 @@ namespace cctz_extension const std::string & name, const std::function(const std::string & name)> & fallback) { - const TimeZone * timezone = auto_time_zones; - while (timezone->name != nullptr) - { - if (timezone->name == name) - break; - ++timezone; - } - if (timezone->size) - return std::make_unique(reinterpret_cast(timezone->data), timezone->size); + std::string_view tz_file = getTimeZone(name.data()); + + if (!tz_file.empty()) + return std::make_unique(tz_file.data(), tz_file.size()); return fallback(name); } diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index b09319c78d6..3d3a3f04941 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -15,8 +15,7 @@ #endif // All timezones present at build time and embedded into ClickHouse binary. -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +extern const char * auto_time_zones[]; namespace { @@ -33,14 +32,14 @@ std::vector allTimezones(bool with_weird_offsets = true) { std::vector result; - const TimeZone * timezone = auto_time_zones; - while (timezone->name) + const auto * timezone_name = auto_time_zones; + while (*timezone_name) { - bool weird_offsets = (std::string_view(timezone->name) == "Africa/Monrovia"); + bool weird_offsets = (std::string_view(*timezone_name) == "Africa/Monrovia"); if (!weird_offsets || with_weird_offsets) - result.push_back(timezone->name); - ++timezone; + result.push_back(*timezone_name); + ++timezone_name; } return result; diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 6b7d1739e33..c3a2e726365 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -30,7 +30,6 @@ endif() add_dependencies(generate-source generate-contributors) set(GENERATED_LICENSES_SRC "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemLicenses.generated.cpp") -set(GENERATED_TIMEZONES_SRC "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemTimeZones.generated.cpp") add_custom_command( OUTPUT StorageSystemLicenses.generated.cpp @@ -38,7 +37,6 @@ add_custom_command( WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) list (APPEND storages_system_sources ${GENERATED_LICENSES_SRC}) -list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC}) # Overlength strings set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w) diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index 41227ab7780..dc3711812a6 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -4,8 +4,7 @@ #include -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +extern const char * auto_time_zones[]; namespace DB { @@ -18,7 +17,7 @@ NamesAndTypesList StorageSystemTimeZones::getNamesAndTypes() void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - for (auto * it = auto_time_zones; it->name != nullptr; ++it) - res_columns[0]->insert(String(it->name)); + for (auto * it = auto_time_zones; *it; ++it) + res_columns[0]->insert(String(*it)); } } From 8013cb1f784f6324b3c7b227499751dc7e666009 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 23 Jul 2023 08:46:44 +0200 Subject: [PATCH 655/871] Remove skip_startup_tables from IDatabase::loadStoredObjects() Signed-off-by: Azat Khuzhin --- src/Databases/DatabaseAtomic.cpp | 5 ++--- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseLazy.cpp | 3 +-- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 9 +-------- src/Databases/DatabaseOrdinary.h | 2 +- src/Databases/DatabaseReplicated.cpp | 5 ++--- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/IDatabase.h | 3 +-- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.h | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Databases/TablesLoader.cpp | 2 +- 14 files changed, 16 insertions(+), 27 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 7e20b6f6535..0f65069db35 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -441,11 +441,10 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin } } -void DatabaseAtomic::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseAtomic::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { beforeLoadingMetadata(local_context, mode); - DatabaseOrdinary::loadStoredObjects(local_context, mode, skip_startup_tables); + DatabaseOrdinary::loadStoredObjects(local_context, mode); } void DatabaseAtomic::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index cb275812098..70553b2d5c2 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index f27c6c0c3ee..896ae99656f 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -37,8 +37,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, } -void DatabaseLazy::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) { diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index b01038073ef..2b1b119754d 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -26,7 +26,7 @@ public: bool canContainDistributedTables() const override { return false; } - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel /*mode*/) override; void createTable( ContextPtr context, diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 8c92b8064ca..51d37b84e14 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -89,8 +89,7 @@ DatabaseOrdinary::DatabaseOrdinary( { } -void DatabaseOrdinary::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, @@ -159,12 +158,6 @@ void DatabaseOrdinary::loadStoredObjects( } pool.wait(); - - if (!skip_startup_tables) - { - /// After all tables was basically initialized, startup them. - startupTables(pool, mode); - } } void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTablesMetadata & metadata, bool is_startup) diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index f9aa3214ef5..cabc8f9c55b 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -21,7 +21,7 @@ public: String getEngineName() const override { return "Ordinary"; } - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; bool supportsLoadingInTopologicalOrder() const override { return true; } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 25c23e2be17..d3b3d4b545f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -495,11 +495,10 @@ void DatabaseReplicated::beforeLoadingMetadata(ContextMutablePtr /*context*/, Lo tryConnectToZooKeeperAndInitDatabase(mode); } -void DatabaseReplicated::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseReplicated::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { beforeLoadingMetadata(local_context, mode); - DatabaseAtomic::loadStoredObjects(local_context, mode, skip_startup_tables); + DatabaseAtomic::loadStoredObjects(local_context, mode); } UInt64 DatabaseReplicated::getMetadataHash(const String & table_name) const diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index ff1a4aba41c..8e33f482ac1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -67,7 +67,7 @@ public: void drop(ContextPtr /*context*/) override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index a9577dfc84a..9bed3c4bfc5 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -134,8 +134,7 @@ public: /// You can call only once, right after the object is created. virtual void loadStoredObjects( /// NOLINT ContextMutablePtr /*context*/, - LoadingStrictnessLevel /*mode*/, - bool /* skip_startup_tables */) + LoadingStrictnessLevel /*mode*/) { } diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 70bd32efed9..94e5ba1773e 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -402,7 +402,7 @@ String DatabaseMySQL::getMetadataPath() const return metadata_path; } -void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) { std::lock_guard lock{mutex}; diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index f34a2fff4f7..e5b1f434d2f 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -76,7 +76,7 @@ public: void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index f4d750f85d4..812a0d8717e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -296,7 +296,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/) } -void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/) { { std::lock_guard lock{mutex}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 31fa036c0ee..d731e06649b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -44,7 +44,7 @@ public: bool empty() const override; - void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index ea0f2072430..f8b4e7fe33b 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -49,7 +49,7 @@ void TablesLoader::loadTables() if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder()) databases_to_load.push_back(database.first); else - database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true); + database.second->loadStoredObjects(global_context, strictness_mode); } if (databases_to_load.empty()) From 282258a855cfed40e0b2cd7c0ada3ec1defe8e06 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 23 Jul 2023 11:29:29 +0200 Subject: [PATCH 656/871] fix style --- src/Common/OptimizedRegularExpression.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index e636b0b987d..05e6aefbb5e 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -439,7 +440,7 @@ catch (...) is_trivial = false; required_substring_is_prefix = false; alternatives.clear(); - std::cerr << "Analyze RegularExpression failed, got error: {}" << DB::getCurrentExceptionMessage(false) << "\n"; + LOG_ERROR(&Poco::Logger::get("OptimizeRegularExpression"), "Analyze RegularExpression failed, got error: {}", DB::getCurrentExceptionMessage(false)); } template From 4c1f8f38cd4073b24064e076a677082db546c680 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 17:30:23 +0200 Subject: [PATCH 657/871] Fix CI --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index e25b5fdbfed..60e6199aaa4 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -147,6 +147,7 @@ function clone_submodules contrib/simdjson contrib/liburing contrib/libfiu + contrib/incbin ) git submodule sync From 8902bbdb60b466498ab2825000502195d5d35c91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 17:59:49 +0200 Subject: [PATCH 658/871] Fix fasttest --- src/Common/FrequencyHolder.cpp | 4 ++++ src/Common/FrequencyHolder.h | 6 ++++++ src/Functions/FunctionsCharsetClassification.cpp | 2 +- src/Functions/FunctionsLanguageClassification.cpp | 4 +--- 4 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp index 3b755cacacb..fe03e6a1b44 100644 --- a/src/Common/FrequencyHolder.cpp +++ b/src/Common/FrequencyHolder.cpp @@ -1,5 +1,7 @@ #include +#if USE_NLP + #include /// Embedded SQL definitions @@ -179,3 +181,5 @@ void FrequencyHolder::loadProgrammingFrequency() } } + +#endif diff --git a/src/Common/FrequencyHolder.h b/src/Common/FrequencyHolder.h index 270e4dbbd2a..73675ed9814 100644 --- a/src/Common/FrequencyHolder.h +++ b/src/Common/FrequencyHolder.h @@ -1,5 +1,9 @@ #pragma once +#include "config.h" + +#if USE_NLP + #include #include @@ -81,3 +85,5 @@ private: EncodingContainer encodings_freq; }; } + +#endif diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index a25da8f6c13..237d4c37fa2 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -3,7 +3,7 @@ #include #include -#include + namespace DB { diff --git a/src/Functions/FunctionsLanguageClassification.cpp b/src/Functions/FunctionsLanguageClassification.cpp index 6088fd52efa..55485d41ce0 100644 --- a/src/Functions/FunctionsLanguageClassification.cpp +++ b/src/Functions/FunctionsLanguageClassification.cpp @@ -5,19 +5,17 @@ #include #include #include -#include #include #include #include -#include #include #include #include #include -#include #include + namespace DB { /* Determine language of Unicode UTF-8 text. From 43bd6d1b8336f282cc4548c0f61b52516f49ac13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 19:00:49 +0300 Subject: [PATCH 659/871] Revert "Add an ability to specify allocations size for sampling memory profiler" --- programs/server/Server.cpp | 21 +++-------- src/Common/MemoryTracker.cpp | 10 +---- src/Common/MemoryTracker.h | 18 --------- src/Core/ServerSettings.h | 8 +--- src/Core/Settings.h | 4 +- src/Interpreters/ProcessList.cpp | 3 -- src/Interpreters/ThreadStatusExt.cpp | 2 - .../__init__.py | 1 - .../configs/max_untracked_memory.xml | 7 ---- .../configs/memory_profiler.xml | 5 --- .../test.py | 37 ------------------- ...r_sample_min_max_allocation_size.reference | 1 - ...profiler_sample_min_max_allocation_size.sh | 18 --------- 13 files changed, 11 insertions(+), 124 deletions(-) delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py delete mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference delete mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 33fdcc9c1a8..9202d4b32c1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1637,26 +1637,17 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) + UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); + if (total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + total_memory_tracker.setProfilerStep(total_memory_profiler_step); } - if (server_settings.total_memory_tracker_sample_probability > 0.0) + double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); + if (total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 52cae0768dc..81cac2617c5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,12 +534,6 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } -bool MemoryTracker::isSizeOkForSampling(UInt64 size) const -{ - /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation - return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); -} - bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 768dc8a7404..4e29d40c953 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,12 +67,6 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; - /// Randomly sample allocations only larger or equal to this size - UInt64 min_allocation_size_bytes = 0; - - /// Randomly sample allocations only smaller or equal to this size - UInt64 max_allocation_size_bytes = 0; - /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -94,8 +88,6 @@ private: void setOrRaiseProfilerLimit(Int64 value); - bool isSizeOkForSampling(UInt64 size) const; - /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -173,16 +165,6 @@ public: sample_probability = value; } - void setSampleMinAllocationSize(UInt64 value) - { - min_allocation_size_bytes = value; - } - - void setSampleMaxAllocationSize(UInt64 value) - { - max_allocation_size_bytes = value; - } - void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f7a6c9e950e..1a9f226041b 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -81,12 +81,8 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ - \ - M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ - M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ - M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) + DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4fc93500910..24be644ee55 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -427,9 +427,7 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ - M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c299572ef41..1503e396298 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,10 +223,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); - thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); - thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); - thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index bac16c05533..398bea26b87 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,8 +83,6 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); - group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); - group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py deleted file mode 100644 index e5a0d9b4834..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/__init__.py +++ /dev/null @@ -1 +0,0 @@ -#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml deleted file mode 100644 index 56fc5ed34ca..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml deleted file mode 100644 index 5b3e17d145f..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml +++ /dev/null @@ -1,5 +0,0 @@ - - 1 - 4096 - 8192 - diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py deleted file mode 100644 index 6ab971fa9c4..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ /dev/null @@ -1,37 +0,0 @@ -from helpers.cluster import ClickHouseCluster -import pytest - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node", - main_configs=["configs/memory_profiler.xml"], - user_configs=["configs/max_untracked_memory.xml"], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_trace_boundaries_work(started_cluster): - node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") - node.query("SYSTEM FLUSH LOGS") - - assert ( - node.query( - "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" - ) - == "1\n" - ) - assert ( - node.query( - "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" - ) - == "0\n" - ) diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh deleted file mode 100755 index b1fbea26da7..00000000000 --- a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings -# requires TraceCollector, does not available under sanitizers and aarch64 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" -${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" - -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" - -# at least something allocated -${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" - -# show wrong allocations -${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From e56e1ebd5d8fbb808867c1f98e421383acf38b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 18:29:07 +0200 Subject: [PATCH 660/871] Fix fasttest --- src/Functions/FunctionsCharsetClassification.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 237d4c37fa2..7704e3eafc0 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -1,4 +1,9 @@ #include + +#include "config.h" + +#if USE_NLP + #include #include @@ -150,3 +155,5 @@ REGISTER_FUNCTION(DetectCharset) } } + +#endif From 039cac69cf6d30cc58c8531b1efac4d9847cb599 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 23 Jul 2023 18:35:37 +0200 Subject: [PATCH 661/871] 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 761b5257a34..86b70f8db70 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 311b3adf89b9d54c4b3bf40feb4179d967ed3d2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:10:41 +0200 Subject: [PATCH 662/871] Fix fasttest --- src/Functions/FunctionsCharsetClassification.cpp | 2 -- src/Functions/FunctionsProgrammingClassification.cpp | 5 +++++ src/Functions/FunctionsTonalityClassification.cpp | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 7704e3eafc0..05b173e3d95 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -1,7 +1,5 @@ #include -#include "config.h" - #if USE_NLP #include diff --git a/src/Functions/FunctionsProgrammingClassification.cpp b/src/Functions/FunctionsProgrammingClassification.cpp index 8a552a30e65..a93e1d9a87d 100644 --- a/src/Functions/FunctionsProgrammingClassification.cpp +++ b/src/Functions/FunctionsProgrammingClassification.cpp @@ -1,4 +1,7 @@ #include + +#if USE_NLP + #include #include #include @@ -118,3 +121,5 @@ REGISTER_FUNCTION(DetectProgrammingLanguage) } } + +#endif diff --git a/src/Functions/FunctionsTonalityClassification.cpp b/src/Functions/FunctionsTonalityClassification.cpp index e39f9c63758..3de38d99c88 100644 --- a/src/Functions/FunctionsTonalityClassification.cpp +++ b/src/Functions/FunctionsTonalityClassification.cpp @@ -1,4 +1,7 @@ #include + +#if USE_NLP + #include #include #include @@ -87,3 +90,5 @@ REGISTER_FUNCTION(DetectTonality) } } + +#endif From 49f4ef6ffb9264d8b4a31c8e4ab683f01afd4268 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:11:24 +0200 Subject: [PATCH 663/871] Fix typo --- src/Functions/FunctionsCharsetClassification.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 05b173e3d95..0a332ab70a9 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -49,7 +49,7 @@ namespace return res; } - /// Сount how many times each bigram occurs in the text. + /// Count how many times each bigram occurs in the text. template ALWAYS_INLINE inline void calculateStats( const UInt8 * data, From e21a4c4c9a3f50436b8e708b6a38cdf8eee3c6be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:57:16 +0200 Subject: [PATCH 664/871] Fix the test --- .../02415_all_new_functions_must_be_documented.reference | 4 ---- .../02415_all_new_functions_must_be_documented.sql | 4 +++- 2 files changed, 3 insertions(+), 5 deletions(-) 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 595ebb483d5..b7097ad329b 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 @@ -238,10 +238,6 @@ defaultValueOfArgumentType defaultValueOfTypeName degrees demangle -detectCharset -detectLanguageUnknown -detectProgrammingLanguage -detectTonality divide dotProduct dumpColumnStructure diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index ed95c06d016..4f40da6c626 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -15,5 +15,7 @@ AND name NOT IN ( 'h3ToGeoBoundary', 'h3ToParent', 'h3ToString', 'h3UnidirectionalEdgeIsValid', 'h3kRing', 'stringToH3', 'geoToS2', 's2CapContains', 's2CapUnion', 's2CellsIntersect', 's2GetNeighbors', 's2RectAdd', 's2RectContains', 's2RectIntersection', 's2RectUnion', 's2ToGeo', 'normalizeUTF8NFC', 'normalizeUTF8NFD', 'normalizeUTF8NFKC', 'normalizeUTF8NFKD', - 'lemmatize', 'tokenize', 'stem', 'synonyms' -- these functions are not enabled in fast test + 'lemmatize', 'tokenize', 'stem', 'synonyms', + 'detectCharset', 'detectLanguageUnknown', 'detectProgrammingLanguage', 'detectTonality' + -- these functions are not enabled in fast test ) ORDER BY name; From 67f643f27e5930765d0b6881c415ffacf369c14f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 21:00:28 +0200 Subject: [PATCH 665/871] Fix error --- contrib/cctz-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7edeada6e59..fde31dd469d 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -55,7 +55,7 @@ foreach(TIMEZONE ${TIMEZONES}) MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} "};\n\n") +file(APPEND ${TIMEZONES_FILE} " nullptr\n};\n\n") file(APPEND ${TIMEZONES_FILE} "#include \n\n") file(APPEND ${TIMEZONES_FILE} "std::string_view getTimeZone(const char * name)\n{\n" ) From e02948580b31c61e32860da04f966a21231e14c7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 23 Jul 2023 22:38:59 +0200 Subject: [PATCH 666/871] Don't shutdown interserver before tables --- programs/server/Server.cpp | 91 +++++++++++++++++++++++++++++--------- programs/server/Server.h | 11 ++++- 2 files changed, 79 insertions(+), 23 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c7a7ba71e83..8c6e41d28c6 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()); + + 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()}); - std::lock_guard lock(servers_lock); for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; @@ -1302,7 +1303,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); @@ -1404,10 +1405,27 @@ try } - for (auto & server : servers_to_start_before_tables) { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); + 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) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } } /// Initialize access storages. @@ -1527,10 +1545,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) @@ -1709,7 +1730,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' " @@ -1967,7 +1988,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, @@ -2189,6 +2209,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) @@ -2237,14 +2274,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(); @@ -2270,11 +2307,19 @@ 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); + + 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.")) @@ -2312,27 +2357,29 @@ 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_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/programs/server/Server.h b/programs/server/Server.h index e9ae6d8d937..d13378dcd65 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -102,6 +102,14 @@ private: 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 @@ private: Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers); + std::vector & servers, + std::vector & servers_to_start_before_tables); }; } From c0f16dcf031b62e2eebdef249c132e9351203bc0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 23 Jul 2023 21:10:12 +0000 Subject: [PATCH 667/871] Test from fuzzer --- .../02831_ast_fuzz_asan_join.reference | 0 .../0_stateless/02831_ast_fuzz_asan_join.sql | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference create mode 100644 tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql diff --git a/tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql new file mode 100644 index 00000000000..7c7bfd2df88 --- /dev/null +++ b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql @@ -0,0 +1,22 @@ +SELECT + '0', + toTypeName(materialize(js2.s)) +FROM +( + SELECT number AS k + FROM numbers(100) +) AS js1 +FULL OUTER JOIN +( + SELECT + toLowCardinality(2147483647 + 256) AS k, + '-0.0000000001', + 1024, + toString(number + 10) AS s + FROM numbers(1024) +) AS js2 ON js1.k = js2.k +ORDER BY + inf DESC NULLS FIRST, + js1.k ASC NULLS LAST, + js2.k ASC +FORMAT `Null` From 1e467867e68c2c382f26291753bab45e2bc87a60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:03:40 +0200 Subject: [PATCH 668/871] Attempt to fix LTO --- contrib/cctz-cmake/CMakeLists.txt | 3 +-- programs/install/Install.cpp | 6 ++++-- programs/server/Server.cpp | 2 +- src/Common/FrequencyHolder.cpp | 6 +++--- src/Common/config.h.in | 4 ++++ src/Server/WebUIRequestHandler.cpp | 8 +++++--- src/Storages/System/attachInformationSchemaTables.cpp | 10 ++++++---- src/configure_config.cmake | 2 ++ 8 files changed, 26 insertions(+), 15 deletions(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index fde31dd469d..7161f743de1 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -44,7 +44,7 @@ file(APPEND ${TIMEZONES_FILE} "#include \n") set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TIMEZONE}\");\n") + file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TZDIR}/${TIMEZONE}\");\n") MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) @@ -71,7 +71,6 @@ file(APPEND ${TIMEZONES_FILE} "}\n") add_library (tzdata ${TIMEZONES_FILE}) target_link_libraries(tzdata ch_contrib::incbin) -target_include_directories(tzdata PRIVATE ${TZDIR}) target_link_libraries(_cctz tzdata) add_library(ch_contrib::cctz ALIAS _cctz) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index da2c95af62c..d7086c95beb 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -34,9 +34,11 @@ #include +#include "config.h" + /// Embedded configuration files used inside the install program -INCBIN(resource_config_xml, "config.xml"); -INCBIN(resource_users_xml, "users.xml"); +INCBIN(resource_config_xml, SOURCE_DIR "/programs/server/config.xml"); +INCBIN(resource_users_xml, SOURCE_DIR "/programs/server/users.xml"); /** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 229a169dc1e..2ab89ad048a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -130,7 +130,7 @@ #include /// A minimal file used when the server is run without installation -INCBIN(resource_embedded_xml, "embedded.xml"); +INCBIN(resource_embedded_xml, SOURCE_DIR "/programs/server/embedded.xml"); namespace CurrentMetrics { diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp index fe03e6a1b44..7dc1f622aeb 100644 --- a/src/Common/FrequencyHolder.cpp +++ b/src/Common/FrequencyHolder.cpp @@ -5,9 +5,9 @@ #include /// Embedded SQL definitions -INCBIN(resource_charset_zst, "charset.zst"); -INCBIN(resource_tonality_ru_zst, "tonality_ru.zst"); -INCBIN(resource_programming_zst, "programming.zst"); +INCBIN(resource_charset_zst, SOURCE_DIR "/contrib/nlp-data/charset.zst"); +INCBIN(resource_tonality_ru_zst, SOURCE_DIR "/contrib/nlp-data/tonality_ru.zst"); +INCBIN(resource_programming_zst, SOURCE_DIR "/contrib/nlp-data/programming.zst"); namespace DB diff --git a/src/Common/config.h.in b/src/Common/config.h.in index a2c18fc330f..628f0847d65 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,3 +59,7 @@ #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE #cmakedefine01 USE_BCRYPT + +/// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. +/// That's why we use absolute paths. +#cmakedefine SOURCE_DIR "@SOURCE_DIR@" diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index cb9e8935d8c..6fa1d65de42 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -11,10 +11,12 @@ #include +#include "config.h" + /// Embedded HTML pages -INCBIN(resource_play_html, "play.html"); -INCBIN(resource_dashboard_html, "dashboard.html"); -INCBIN(resource_uplot_js, "js/uplot.js"); +INCBIN(resource_play_html, SOURCE_DIR "/programs/server/play.html"); +INCBIN(resource_dashboard_html, SOURCE_DIR "/programs/server/dashboard.html"); +INCBIN(resource_uplot_js, SOURCE_DIR "/programs/server/js/uplot.js"); namespace DB diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index bfc5c8c64e2..12cef89b553 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -5,11 +5,13 @@ #include #include +#include "config.h" + /// Embedded SQL definitions -INCBIN(resource_schemata_sql, "schemata.sql"); -INCBIN(resource_tables_sql, "tables.sql"); -INCBIN(resource_views_sql, "views.sql"); -INCBIN(resource_columns_sql, "columns.sql"); +INCBIN(resource_schemata_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/schemata.sql"); +INCBIN(resource_tables_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/tables.sql"); +INCBIN(resource_views_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/views.sql"); +INCBIN(resource_columns_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/columns.sql"); namespace DB diff --git a/src/configure_config.cmake b/src/configure_config.cmake index ae6305705c2..5529e2f2f39 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -162,3 +162,5 @@ endif () if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() + +set(SOURCE_DIR ${CMAKE_SOURCE_DIR}) From cfed3589b03961252adcfef746576ca717c7246f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 23 Jul 2023 20:25:44 +0300 Subject: [PATCH 669/871] fix style --- .../integration/helpers/s3_mocks/broken_s3.py | 31 ++++++----- .../test_checking_s3_blobs_paranoid/test.py | 54 ++++++++++++------- 2 files changed, 53 insertions(+), 32 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 0c794ae1ad4..6e1572af262 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -37,7 +37,9 @@ class MockControl: ) assert response == "OK", response - def setup_action(self, when, count=None, after=None, action="error_500", action_args=None): + def setup_action( + self, when, count=None, after=None, action="error_500", action_args=None + ): url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: @@ -175,7 +177,7 @@ class _ServerRuntime: class ConnectionResetByPeerAction: def __init__(self, with_partial_data=None): self.partial_data = "" - if with_partial_data is not None: + if with_partial_data is not None and with_partial_data == "1": self.partial_data = ( '\n' "\n" @@ -193,9 +195,7 @@ class _ServerRuntime: time.sleep(1) request_handler.connection.setsockopt( - socket.SOL_SOCKET, - socket.SO_LINGER, - struct.pack('ii', 1, 0) + socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 1, 0) ) request_handler.connection.close() @@ -211,7 +211,9 @@ class _ServerRuntime: if self.action == "connection_refused": self.error_handler = _ServerRuntime.ConnectionRefusedAction() elif self.action == "connection_reset_by_peer": - self.error_handler = _ServerRuntime.ConnectionResetByPeerAction(*self.action_args) + self.error_handler = _ServerRuntime.ConnectionResetByPeerAction( + *self.action_args + ) elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) else: @@ -311,9 +313,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.read_all_input() self.send_response(307) - url = ( - f"http://{host}:{port}{self.path}" - ) + url = f"http://{host}:{port}{self.path}" self.log_message("redirect to %s", url) self.send_header("Location", url) self.end_headers() @@ -403,7 +403,9 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params( + params + ) self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() @@ -433,8 +435,13 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_create_multi_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) - self.log_message("set at_create_multi_part_upload %s", _runtime.at_create_multi_part_upload) + _runtime.at_create_multi_part_upload = ( + _ServerRuntime.CountAfter.from_cgi_params(params) + ) + self.log_message( + "set at_create_multi_part_upload %s", + _runtime.at_create_multi_part_upload, + ) return self._ok() if path[1] == "reset": diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index b6b598c4557..5f7a2a6cc7f 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -172,9 +172,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert count_s3_errors >= 2 -def test_when_s3_connection_refused_at_write_retried( - cluster, broken_s3 -): +def test_when_s3_connection_refused_is_retried(cluster, broken_s3): node = cluster.instances["node"] broken_s3.setup_fake_multpartuploads() @@ -231,11 +229,14 @@ def test_when_s3_connection_refused_at_write_retried( ) assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error, error + assert ( + "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error + ), error -def test_when_s3_connection_reset_by_peer_at_write_retried( - cluster, broken_s3 +@pytest.mark.parametrize("send_something", [True, False]) +def test_when_s3_connection_reset_by_peer_at_upload_is_retried( + cluster, broken_s3, send_something ): node = cluster.instances["node"] @@ -244,14 +245,15 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( count=3, after=2, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_upload_is_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -279,13 +281,14 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( count=1000, after=2, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED_1" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_upload_is_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -301,12 +304,17 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( query_id=insert_query_id, ) - assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Connection reset by peer." in error + or + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + ), error -def test_when_s3_connection_reset_by_peer_at_read_retried( - cluster, broken_s3 +@pytest.mark.parametrize("send_something", [True, False]) +def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( + cluster, broken_s3, send_something ): node = cluster.instances["node"] @@ -315,14 +323,15 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( count=3, after=0, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -350,14 +359,15 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( count=1000, after=0, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED_1" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -373,5 +383,9 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( query_id=insert_query_id, ) - assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Connection reset by peer." in error + or + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + ), error From 7b4d0cf9d5b261eb68bd1db4021fcc350b907fc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:51:20 +0200 Subject: [PATCH 670/871] Fix Darwin --- contrib/incbin-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt index e64ebc99c73..8f4dad7e0d9 100644 --- a/contrib/incbin-cmake/CMakeLists.txt +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -2,3 +2,7 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/incbin") add_library(_incbin INTERFACE) target_include_directories(_incbin SYSTEM INTERFACE ${LIBRARY_DIR}) add_library(ch_contrib::incbin ALIAS _incbin) + +# Warning "incbin is incompatible with bitcode. Using the library will break upload to App Store if you have bitcode enabled. +# Add `#define INCBIN_SILENCE_BITCODE_WARNING` before including this header to silence this warning." +target_compile_definitions(_inclin PUBLIC INCBIN_SILENCE_BITCODE_WARNING) From 641c086dbd771c14cc7db089e265ec508da9ccff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:53:11 +0200 Subject: [PATCH 671/871] Fix Darwin --- contrib/incbin-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt index 8f4dad7e0d9..5778cf83c22 100644 --- a/contrib/incbin-cmake/CMakeLists.txt +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -5,4 +5,4 @@ add_library(ch_contrib::incbin ALIAS _incbin) # Warning "incbin is incompatible with bitcode. Using the library will break upload to App Store if you have bitcode enabled. # Add `#define INCBIN_SILENCE_BITCODE_WARNING` before including this header to silence this warning." -target_compile_definitions(_inclin PUBLIC INCBIN_SILENCE_BITCODE_WARNING) +target_compile_definitions(_incbin INTERFACE INCBIN_SILENCE_BITCODE_WARNING) From 40f5649811bb579b3cf8d634281f862675934773 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 01:02:11 +0200 Subject: [PATCH 672/871] Fix test --- .../no_allow_vertical_merges_from_compact_to_wide_parts.xml | 5 +++++ .../test_vertical_merges_from_compact_parts.py | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml diff --git a/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml b/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml new file mode 100644 index 00000000000..c69be846c46 --- /dev/null +++ b/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml @@ -0,0 +1,5 @@ + + + 0 + + diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 82ffcc20b60..481621cacfe 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -15,7 +15,7 @@ node_old = cluster.add_instance( ) node_new = cluster.add_instance( "node2", - main_configs=["configs/no_compress_marks.xml"], + main_configs=["configs/no_compress_marks.xml", "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml"], with_zookeeper=True, stay_alive=True, allow_analyzer=False, From dba7a0dffc4927a88c04cb7b9ec93faeeba40b3c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 23 Jul 2023 23:18:19 +0000 Subject: [PATCH 673/871] Automatic style fix --- .../test_vertical_merges_from_compact_parts.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 481621cacfe..9c9d1a4d312 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -15,7 +15,10 @@ node_old = cluster.add_instance( ) node_new = cluster.add_instance( "node2", - main_configs=["configs/no_compress_marks.xml", "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml"], + main_configs=[ + "configs/no_compress_marks.xml", + "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml", + ], with_zookeeper=True, stay_alive=True, allow_analyzer=False, From f46568929dcecf5851ae6290bb3be4fc64e52308 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 24 Jul 2023 07:35:48 +0800 Subject: [PATCH 674/871] add doc --- docs/en/operations/system-tables/events.md | 1 + docs/en/operations/system-tables/metrics.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index ba5602ee292..b53628a8384 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -10,6 +10,7 @@ Columns: - `event` ([String](../../sql-reference/data-types/string.md)) — Event name. - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for event. **Example** diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..ea36411cf3d 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -10,6 +10,7 @@ Columns: - `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for metric. The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. From d7cdfb47d3795a3a09c2a204789c95e9726dc2b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 01:56:04 +0200 Subject: [PATCH 675/871] Fix merge --- src/IO/WriteHelpers.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 0494cdf22e7..76778543bd0 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -953,6 +953,11 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { auto remainder = value % 10; value /= 10; + + if (remainder != 0 && last_nonzero_pos == 0) + last_nonzero_pos = pos; + + buf[pos] += static_cast(remainder); } writeChar('.', ostr); From 75efee9675f277fc3405ca5b256296aa406baca4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:34:00 +0200 Subject: [PATCH 676/871] Fix errors --- programs/install/CMakeLists.txt | 3 --- programs/server/CMakeLists.txt | 2 +- src/CMakeLists.txt | 3 +-- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index f3f562bab7c..c3f4d96d631 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -10,6 +10,3 @@ set (CLICKHOUSE_INSTALL_LINK ) clickhouse_program_add_library(install) - -# For incbin -target_include_directories(clickhouse-install-lib PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/../server") diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index e008e65acf6..b8241afa1eb 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -29,4 +29,4 @@ endif() clickhouse_program_add(server) -target_include_directories(clickhouse-server-lib PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) +install(FILES config.xml users.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-server" COMPONENT clickhouse) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fda8bafde59..975bf9bb618 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -296,7 +296,7 @@ macro (dbms_target_include_directories) endforeach () endmacro () -dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src" "${ClickHouse_SOURCE_DIR}/programs/server") +dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") if (TARGET ch_contrib::llvm) @@ -561,7 +561,6 @@ if (ENABLE_NLP) dbms_target_link_libraries (PUBLIC ch_contrib::stemmer) dbms_target_link_libraries (PUBLIC ch_contrib::wnb) dbms_target_link_libraries (PUBLIC ch_contrib::lemmagen) - target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_SOURCE_DIR}/contrib/nlp-data) endif() if (TARGET ch_contrib::ulid) From 169b9d5cc0c8dc54d31bc7229204b195f294c877 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:49:06 +0200 Subject: [PATCH 677/871] 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 da1172c8916..aaaeeb7339d 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 678/871] 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 d07a6521544..fc3ea3a13ca 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 5d8f8ca8741..c118fccded4 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 76641b656a2..701e02a85ac 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -254,6 +254,9 @@ public: /// 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 41fc4657854..5e6b043c31c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -434,6 +434,8 @@ public: 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 08fbb61960f..e6fcbc203e6 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -41,6 +41,8 @@ public: 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 679/871] 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 3dea52faf46..5efba383e4b 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 680/871] 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 4143014a7b3..e6d72ddf17b 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 681/871] 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 0cf3f360994..953e38d56cd 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 030c0b77dd5..1cc5a3b1a77 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -50,6 +50,8 @@ public: { /// 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 5cc13f45df4..7a99c363232 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 d4005d81f1b..ac869cd89f9 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 682/871] 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 5efba383e4b..2fed626ffb7 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 41f6da55686..511914e99e4 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 e5fc01be9f4..cbed05e30ed 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 683/871] 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 de10a445d01..b90acd1d576 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -144,12 +144,6 @@ public: 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 684/871] Split StorageReplicatedMergeTree reading methods Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.cpp | 141 ++++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 32 ++++- 2 files changed, 119 insertions(+), 54 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 13c0fb3f7c2..4e053c4598c 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(); + + /// 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); if (local_context->canUseParallelReplicasOnInitiator()) + return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + + readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); +} + +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); +} + +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 parallel_replicas_cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + + ASTPtr modified_query_ast; + Block header; + if (local_context->getSettingsRef().allow_experimental_analyzer) { - auto table_id = getStorageID(); + auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); - ASTPtr modified_query_ast; - - Block header; - - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); - - 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(); - } - - auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); - - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( - header, - {}, - storage_snapshot, - processed_stage); - - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, - select_stream_factory, modified_query_ast, - local_context, query_info, cluster); + 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 1a1b3c3b10c..ded940bc1d2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -130,7 +130,7 @@ public: 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 @@ private: 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 b22313ef2d721ec0f8687515de58f4e2ba785d1d Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 24 Jul 2023 03:54:34 +0000 Subject: [PATCH 685/871] Replace with three way comparison --- src/Common/IntervalTree.h | 34 ++-------------------------------- 1 file changed, 2 insertions(+), 32 deletions(-) diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index 2214a4e842d..ad079a312f2 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -27,39 +27,9 @@ struct Interval }; template -bool operator<(const Interval & lhs, const Interval & rhs) +auto operator<=>(const Interval & lhs, const Interval & rhs) { - return std::tie(lhs.left, lhs.right) < std::tie(rhs.left, rhs.right); -} - -template -bool operator<=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) <= std::tie(rhs.left, rhs.right); -} - -template -bool operator==(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) == std::tie(rhs.left, rhs.right); -} - -template -bool operator!=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) != std::tie(rhs.left, rhs.right); -} - -template -bool operator>(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) > std::tie(rhs.left, rhs.right); -} - -template -bool operator>=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) >= std::tie(rhs.left, rhs.right); + return std::tie(lhs.left, lhs.right) <=> std::tie(rhs.left, rhs.right); } struct IntervalTreeVoidValue From ac54be9652414e10a1b79ec4f92439db5155310b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:56:18 +0200 Subject: [PATCH 686/871] 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 fa24b146fec..c86c3ba0ab2 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 2389e0f0b68d03ecbb117745ed00c54979715ea7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 09:54:51 +0200 Subject: [PATCH 687/871] Randomize timezone in tests across non-deterministic around 1970 and default There was some cases when some patches to the datetime code leads to flaky tests, due to the tests itself had been runned using regular timezone (TZ). But if you will this tests with something "specific" (that is not strictly defined around 1970 year), those tests will fail. So to catch such issues in the PRs itself, let's randomize session_timezone as well. Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 3 +++ tests/clickhouse-test | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index fe53925ecc8..3694fb7c2f6 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -4,6 +4,9 @@ set -e -x -a # Choose random timezone for this test run. +# +# NOTE: that clickhouse-test will randomize session_timezone by itself as well +# (it will choose between default server timezone and something specific). TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" echo "Choosen random timezone $TZ" ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone diff --git a/tests/clickhouse-test b/tests/clickhouse-test index abd109d00b2..185e3003c95 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -529,6 +529,12 @@ def threshold_generator(always_on_prob, always_off_prob, min_val, max_val): return gen +# To keep dependency list as short as possible, tzdata is not used here (to +# avoid try/except block for import) +def get_localzone(): + return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) + + class SettingsRandomizer: settings = { "max_insert_threads": lambda: 0 @@ -602,6 +608,19 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), + "session_timezone": lambda: random.choice( + [ + # special non-deterministic around 1970 timezone, see [1]. + # + # [1]: https://github.com/ClickHouse/ClickHouse/issues/42653 + "America/Mazatlan", + "America/Hermosillo", + "Mexico/BajaSur", + # server default that is randomized across all timezones + # NOTE: due to lots of trickery we cannot use empty timezone here, but this should be the same. + get_localzone(), + ] + ), } @staticmethod From bc167dfde81c44bb93ee7dd0c634ff3428ea3c33 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 06:20:05 +0200 Subject: [PATCH 688/871] clickhouse-test: add proper escaping for HTTP parameters The problem is that old versions of cURL (7.81.0 at least) handle additional parameters incorrectly if in previous parameter was "/": $ docker run --rm curlimages/curl:8.1.2 --http1.1 --get -vvv 'http://kernel.org/?bar=foo/baz' --data-urlencode "query=select 1 format Null"; echo > GET /?bar=foo/baz&query=select+1+format+Null HTTP/1.1 > User-Agent: curl/8.1.2 $ docker run --rm curlimages/curl:7.81.0 --http1.1 --get -vvv 'http://kernel.org/?bar=foo/baz' --data-urlencode "query=select 1 format Null"; echo > GET /?bar=foo/baz?query=select+1+format+Null HTTP/1.1 > User-Agent: curl/7.81.0-DEV Note, that I thought about making the same for cli, but it is not that easy, even after getting rid of sh -c and string contantenation, it still cannot be done for CLICKHOUSE_CLIENT_OPT. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 185e3003c95..c63e1e3ae52 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -625,16 +625,16 @@ class SettingsRandomizer: @staticmethod def get_random_settings(args): - random_settings = [] + random_settings = {} is_debug = BuildFlags.DEBUG in args.build_flags for setting, generator in SettingsRandomizer.settings.items(): if ( is_debug and setting == "allow_prefetched_read_pool_for_remote_filesystem" ): - random_settings.append(f"{setting}=0") + random_settings[setting] = 0 else: - random_settings.append(f"{setting}={generator()}") + random_settings[setting] = generator() return random_settings @@ -670,10 +670,10 @@ class MergeTreeSettingsRandomizer: @staticmethod def get_random_settings(args): - random_settings = [] + random_settings = {} for setting, generator in MergeTreeSettingsRandomizer.settings.items(): if setting not in args.changed_merge_tree_settings: - random_settings.append(f"{setting}={generator()}") + random_settings[setting] = generator() return random_settings @@ -785,7 +785,14 @@ class TestCase: @staticmethod def cli_format_settings(settings_list) -> str: - return " ".join([f"--{setting}" for setting in settings_list]) + out = [] + for k, v in settings_list.items(): + out.extend([f"--{k}", str(v)]) + return " ".join(out) + + @staticmethod + def http_format_settings(settings_list) -> str: + return urllib.parse.urlencode(settings_list) def has_show_create_table_in_test(self): return not subprocess.call(["grep", "-iq", "show create", self.case_file]) @@ -793,11 +800,12 @@ class TestCase: def add_random_settings(self, client_options): new_options = "" if self.randomize_settings: + http_params = self.http_format_settings(self.random_settings) if len(self.base_url_params) == 0: - os.environ["CLICKHOUSE_URL_PARAMS"] = "&".join(self.random_settings) + os.environ["CLICKHOUSE_URL_PARAMS"] = http_params else: os.environ["CLICKHOUSE_URL_PARAMS"] = ( - self.base_url_params + "&" + "&".join(self.random_settings) + self.base_url_params + "&" + http_params ) new_options += f" {self.cli_format_settings(self.random_settings)}" From 6ae4d291800c7d9b32622f1d520f1ab27b9f90b7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 13:22:31 +0200 Subject: [PATCH 689/871] Fix tests after session_timezone randomization Signed-off-by: Azat Khuzhin --- .../0_stateless/00387_use_client_time_zone.sh | 3 ++- tests/queries/0_stateless/00427_alter_primary_key.sh | 11 ++++++----- tests/queries/0_stateless/00933_ttl_simple.sql | 12 ++++++++++++ ...42_system_reload_dictionary_reloads_completely.sh | 4 ++-- .../0_stateless/01070_modify_ttl_recalc_only.sql | 3 +++ .../0_stateless/02530_dictionaries_update_field.sh | 3 ++- 6 files changed, 27 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00387_use_client_time_zone.sh b/tests/queries/0_stateless/00387_use_client_time_zone.sh index 2a6d81eebfe..e54d5244eef 100755 --- a/tests/queries/0_stateless/00387_use_client_time_zone.sh +++ b/tests/queries/0_stateless/00387_use_client_time_zone.sh @@ -5,4 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -env TZ=UTC ${CLICKHOUSE_CLIENT} --use_client_time_zone=1 --query="SELECT toDateTime(1000000000)" +# NOTE: session_timezone overrides use_client_time_zone, disable it randomization +env TZ=UTC ${CLICKHOUSE_CLIENT} --session_timezone '' --use_client_time_zone=1 --query="SELECT toDateTime(1000000000)" diff --git a/tests/queries/0_stateless/00427_alter_primary_key.sh b/tests/queries/0_stateless/00427_alter_primary_key.sh index 1269e2ad6e3..f9984384d79 100755 --- a/tests/queries/0_stateless/00427_alter_primary_key.sh +++ b/tests/queries/0_stateless/00427_alter_primary_key.sh @@ -7,11 +7,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function perform() { local query=$1 - TZ=UTC $CLICKHOUSE_CLIENT \ - --allow_deprecated_syntax_for_merge_tree=1 \ - --use_client_time_zone=1 \ - --input_format_values_interpret_expressions=0 \ - --query "$query" 2>/dev/null + local settings=( + --allow_deprecated_syntax_for_merge_tree 1 + --session_timezone UTC + --input_format_values_interpret_expressions 0 + ) + TZ=UTC $CLICKHOUSE_CLIENT "${settings[@]}" --query "$query" 2>/dev/null if [ "$?" -ne 0 ]; then echo "query failed" fi diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 2bf686822d5..ad40e7c7e47 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -1,3 +1,15 @@ +-- disable timezone randomization since otherwise TTL may fail at particular datetime, i.e.: +-- +-- SELECT +-- now(), +-- toDate(toTimeZone(now(), 'America/Mazatlan')), +-- today() +-- +-- ┌───────────────now()─┬─toDate(toTimeZone(now(), 'America/Mazatlan'))─┬────today()─┐ +-- │ 2023-07-24 06:24:06 │ 2023-07-23 │ 2023-07-24 │ +-- └─────────────────────┴───────────────────────────────────────────────┴────────────┘ +set session_timezone = ''; + drop table if exists ttl_00933_1; -- Column TTL works only with wide parts, because it's very expensive to apply it for compact parts diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index f2b30e05040..9d34470c38d 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -7,8 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail -# Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +# NOTE: dictionaries TTLs works with server timezone, so session_timeout cannot be used +$CLICKHOUSE_CLIENT --session_timezone '' --multiquery <<'EOF' DROP DATABASE IF EXISTS dictdb_01042; CREATE DATABASE dictdb_01042; CREATE TABLE dictdb_01042.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index 247e412484f..7ac70d41871 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -2,6 +2,9 @@ set mutations_sync = 2; +-- system.parts has server default, timezone cannot be randomized +set session_timezone = ''; + drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 569466fe606..6ac10ea2308 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -5,7 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q " +# NOTE: dictionaries will be updated according to server TZ, not session, so prohibit it's randomization +$CLICKHOUSE_CLIENT --session_timezone '' -q " CREATE TABLE table_for_update_field_dictionary ( key UInt64, From 810137e57a53467e9fea668769749c559af12bc1 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 24 Jul 2023 05:59:07 +0000 Subject: [PATCH 690/871] 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 3a7f6d4d854..37821f0fee1 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 b8c5ee77f0c..981f1c7b5a2 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 c7a0f355a92..f84768beccc 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 691/871] 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 1fc0e3adf96..a48d8d47489 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 692/871] 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 00000000000..9daeafb9864 --- /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 00000000000..8d316d3413f --- /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 693/871] 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 a48d8d47489..79168d82c54 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 694/871] 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 9daeafb9864..ea545c90391 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 8d316d3413f..0e58c716c9f 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 695/871] 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 d0ce2e03016..a863a6e2047 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 96d40ff3c4dd34a9396c625b8a1d57f697f80dd0 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 24 Jul 2023 07:30:32 +0000 Subject: [PATCH 696/871] fix --- src/Common/IntervalTree.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index ad079a312f2..9a42aadf70e 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -32,6 +32,12 @@ auto operator<=>(const Interval & lhs, const Interval std::tie(rhs.left, rhs.right); } +template +bool operator==(const Interval & lhs, const Interval & rhs) +{ + return std::tie(lhs.left, lhs.right) == std::tie(rhs.left, rhs.right); +} + struct IntervalTreeVoidValue { }; From 0401dc453e9502697328879728bf0dbf7c1dd9e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Jul 2023 10:14:23 +0200 Subject: [PATCH 697/871] Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop Since there can be some leftovers: 2023.07.24 07:08:25.238066 [ 140 ] {} Application: Code: 219. DB::Exception: Cannot drop: filesystem error: in remove: Directory not empty ["/var/lib/clickhouse/data/system/"]. Probably database contain some detached tables or metadata leftovers from Ordinary engine. If you want to remove all data anyway, try to attach database back and drop it again with enabled force_remove_data_recursively_on_drop setting: Exception while trying to convert database system from Ordinary to Atomic. It may be in some intermediate state. You can finish conversion manually by moving the rest tables from system to .tmp_convert.system.9396432095832455195 (using RENAME TABLE) and executing DROP DATABASE system and RENAME DATABASE .tmp_convert.system.9396432095832455195 TO system. (DATABASE_NOT_EMPTY), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e68af57 in /usr/bin/clickhouse 1. ? @ 0x000000000cab443c in /usr/bin/clickhouse 2. DB::DatabaseOnDisk::drop(std::shared_ptr) @ 0x000000001328d617 in /usr/bin/clickhouse 3. DB::DatabaseCatalog::detachDatabase(std::shared_ptr, String const&, bool, bool) @ 0x0000000013524a6c in /usr/bin/clickhouse 4. DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::shared_ptr&, std::vector, DB::UUIDTag>, std::allocator, DB::UUIDTag>>>&) @ 0x0000000013bc05e4 in /usr/bin/clickhouse 5. DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) @ 0x0000000013bbc6b8 in /usr/bin/clickhouse 6. DB::InterpreterDropQuery::execute() @ 0x0000000013bbba22 in /usr/bin/clickhouse 7. ? @ 0x00000000140b13a5 in /usr/bin/clickhouse 8. DB::executeQuery(String const&, std::shared_ptr, bool, DB::QueryProcessingStage::Enum) @ 0x00000000140ad20e in /usr/bin/clickhouse 9. ? @ 0x00000000140d2ef0 in /usr/bin/clickhouse 10. DB::maybeConvertSystemDatabase(std::shared_ptr) @ 0x00000000140d0aaf in /usr/bin/clickhouse 11. DB::Server::main(std::vector> const&) @ 0x000000000e724e55 in /usr/bin/clickhouse 12. Poco::Util::Application::run() @ 0x0000000017ead086 in /usr/bin/clickhouse 13. DB::Server::run() @ 0x000000000e714a5d in /usr/bin/clickhouse 14. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000017ec07b9 in /usr/bin/clickhouse 15. mainEntryClickHouseServer(int, char**) @ 0x000000000e711a26 in /usr/bin/clickhouse 16. main @ 0x0000000008cf13cf in /usr/bin/clickhouse 17. __libc_start_main @ 0x0000000000021b97 in /lib/x86_64-linux-gnu/libc-2.27.so 18. _start @ 0x00000000080705ae in /usr/bin/clickhouse (version 23.7.1.2012) Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 9 +++++++++ .../force_remove_data_recursively_on_drop.xml | 7 +++++++ .../test_version_update_after_mutation/test.py | 13 ++++++++++--- 3 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eff44de842a..0448eb2437f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3199,6 +3199,7 @@ class ClickHouseInstance: ): self.name = name self.base_cmd = cluster.base_cmd + self.base_dir = base_path self.docker_id = cluster.get_instance_docker_id(self.name) self.cluster = cluster self.hostname = hostname if hostname is not None else self.name @@ -4193,6 +4194,14 @@ class ClickHouseInstance: ["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"] ) + def put_users_config(self, config_path): + """Put new config (useful if you cannot put it at the start)""" + + instance_config_dir = p.abspath(p.join(self.path, "configs")) + users_d_dir = p.abspath(p.join(instance_config_dir, "users.d")) + config_path = p.join(self.base_dir, config_path) + shutil.copy(config_path, users_d_dir) + def create_dir(self): """Create the instance directory and all the needed files there.""" diff --git a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml new file mode 100644 index 00000000000..7a00648b28e --- /dev/null +++ b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index c80205d48c1..416220c93c3 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -51,6 +51,12 @@ def start_cluster(): cluster.shutdown() +def restart_node(node): + # set force_remove_data_recursively_on_drop (cannot be done before, because the version is too old) + node.put_users_config("configs/force_remove_data_recursively_on_drop.xml") + node.restart_with_latest_version(signal=9, fix_metadata=True) + + def test_mutate_and_upgrade(start_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS mt") @@ -67,8 +73,9 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader - node1.restart_with_latest_version(signal=9, fix_metadata=True) - node2.restart_with_latest_version(signal=9, fix_metadata=True) + + restart_node(node1) + restart_node(node2) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -124,7 +131,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - node3.restart_with_latest_version(signal=9, fix_metadata=True) + restart_node(node3) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From efa638ef3cc7db3c6149b7c031cc4c7904987abd Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Wed, 19 Jul 2023 12:53:27 +0200 Subject: [PATCH 698/871] MaterializedMySQL: Support unquoted utf-8 strings in DDL MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Since ClickHouse does not support unquoted utf-8 strings but MySQL does. Instead of fixing Lexer to recognize utf-8 chars as TokenType::BareWord, suggesting to quote all unrecognized tokens before applying any DDL. Actual parsing and validating the syntax will be done by particular Parser. If there is any TokenType::Error, the query is unable to be parsed anyway. Quoting such tokens can provide the support of utf-8 names. See `tryQuoteUnrecognizedTokens` and `QuoteUnrecognizedTokensTest`. mysql> CREATE TABLE 道.渠(... is converted to CREATE TABLE `道`.`渠`(... Also fixed the bug with missing * while doing SELECT in full sync because db or table name are back quoted when not needed. --- src/Common/quoteString.cpp | 11 + src/Common/quoteString.h | 3 + .../MySQL/MaterializedMySQLSyncThread.cpp | 7 +- .../gtest_try_quote_unrecognized_tokens.cpp | 289 ++++++++++++++++++ .../MySQL/tryQuoteUnrecognizedTokens.cpp | 96 ++++++ .../MySQL/tryQuoteUnrecognizedTokens.h | 10 + src/Storages/StorageMySQL.cpp | 11 +- .../materialized_with_ddl.py | 122 ++++++++ .../test_materialized_mysql_database/test.py | 6 + 9 files changed, 542 insertions(+), 13 deletions(-) create mode 100644 src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp create mode 100644 src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp create mode 100644 src/Databases/MySQL/tryQuoteUnrecognizedTokens.h diff --git a/src/Common/quoteString.cpp b/src/Common/quoteString.cpp index b464f4837a1..17129441c8f 100644 --- a/src/Common/quoteString.cpp +++ b/src/Common/quoteString.cpp @@ -44,4 +44,15 @@ String backQuoteIfNeed(StringRef x) return res; } + +String backQuoteMySQL(StringRef x) +{ + String res(x.size, '\0'); + { + WriteBufferFromString wb(res); + writeBackQuotedStringMySQL(x, wb); + } + return res; +} + } diff --git a/src/Common/quoteString.h b/src/Common/quoteString.h index b83988258e2..3f17d6e7621 100644 --- a/src/Common/quoteString.h +++ b/src/Common/quoteString.h @@ -24,4 +24,7 @@ String backQuote(StringRef x); /// Quote the identifier with backquotes, if required. String backQuoteIfNeed(StringRef x); +/// Quote the identifier with backquotes, for use in MySQL queries. +String backQuoteMySQL(StringRef x); + } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 603bf3d0166..673bd155f77 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -342,9 +343,8 @@ static inline String rewriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, { std::make_shared(), "column_type" } }; - const String & query = "SELECT COLUMN_NAME AS column_name, COLUMN_TYPE AS column_type FROM INFORMATION_SCHEMA.COLUMNS" - " WHERE TABLE_SCHEMA = '" + backQuoteIfNeed(database_name) + - "' AND TABLE_NAME = '" + backQuoteIfNeed(table_name) + "' ORDER BY ORDINAL_POSITION"; + String query = "SELECT COLUMN_NAME AS column_name, COLUMN_TYPE AS column_type FROM INFORMATION_SCHEMA.COLUMNS" + " WHERE TABLE_SCHEMA = '" + database_name + "' AND TABLE_NAME = '" + table_name + "' ORDER BY ORDINAL_POSITION"; StreamSettings mysql_input_stream_settings(global_settings, false, true); auto mysql_source = std::make_unique(connection, query, tables_columns_sample_block, mysql_input_stream_settings); @@ -812,6 +812,7 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even CurrentThread::QueryScope query_scope(query_context); String query = query_event.query; + tryQuoteUnrecognizedTokens(query, query); if (!materialized_tables_list.empty()) { auto table_id = tryParseTableIDFromDDL(query, query_event.schema); diff --git a/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp b/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp new file mode 100644 index 00000000000..9c76deb2712 --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp @@ -0,0 +1,289 @@ +#include + +#include + +using namespace DB; + +struct TestCase +{ + String query; + String res; + bool ok; + + TestCase( + const String & query_, + const String & res_, + bool ok_) + : query(query_) + , res(res_) + , ok(ok_) + { + } +}; + +std::ostream & operator<<(std::ostream & ostr, const TestCase & test_case) +{ + return ostr << '"' << test_case.query << "\" -> \"" << test_case.res << "\" ok:" << test_case.ok; +} + +class QuoteUnrecognizedTokensTest : public ::testing::TestWithParam +{ +}; + +TEST_P(QuoteUnrecognizedTokensTest, escape) +{ + const auto & [query, expected, ok] = GetParam(); + String actual; + bool res = tryQuoteUnrecognizedTokens(query, actual); + EXPECT_EQ(ok, res); + EXPECT_EQ(expected, actual); +} + +INSTANTIATE_TEST_SUITE_P(MaterializedMySQL, QuoteUnrecognizedTokensTest, ::testing::ValuesIn(std::initializer_list{ + { + "", + "", + false + }, + { + "test '\"`", + "", + false + }, + { + "SELECT * FROM db.`table`", + "", + false + }, + { + "道渠", + "`道渠`", + true + }, + { + "道", + "`道`", + true + }, + { + "道道(skip) 道(", + "`道道`(skip) `道`(", + true + }, + { + "`道渠`", + "", + false + }, + { + "'道'", + "", + false + }, + { + "\"道\"", + "", + false + }, + { + "` 道 test 渠 `", + "", + false + }, + { + "skip 道 skip 123", + "skip `道` skip 123", + true + }, + { + "skip 123 `道` skip", + "", + false + }, + { + "skip `道 skip 123", + "", + false + }, + { + "skip test道 skip", + "skip `test道` skip", + true + }, + { + "test道2test", + "`test道2test`", + true + }, + { + "skip test道2test 123", + "skip `test道2test` 123", + true + }, + { + "skip 您a您a您a a您a您a您a 1您2您3您4 skip", + "skip `您a您a您a` `a您a您a您a` `1您2您3您4` skip", + true + }, + { + "skip 您a 您a您a b您2您c您4 skip", + "skip `您a` `您a您a` `b您2您c您4` skip", + true + }, + { + "123您a skip 56_您a 您a2 b_您2_您c123您_a4 skip", + "`123您a` skip `56_您a` `您a2` `b_您2_您c123您_a4` skip", + true + }, + { + "_您_ 123 skip 56_您_您_您_您_您_您_您_您_您_a 您a2 abc 123_您_您_321 a1b2c3 aaaaa您您_a4 skip", + "`_您_` 123 skip `56_您_您_您_您_您_您_您_您_您_a` `您a2` abc `123_您_您_321` a1b2c3 `aaaaa您您_a4` skip", + true + }, + { + "TABLE 您2 您(", + "TABLE `您2` `您`(", + true + }, + { + "TABLE 您.a您2(日2日2 INT", + "TABLE `您`.`a您2`(`日2日2` INT", + true + }, + { + "TABLE 您$.a_您2a_($日2日_2 INT, 您Hi好 a您b好c)", + "TABLE `您`$.`a_您2a_`($`日2日_2` INT, `您Hi好` `a您b好c`)", + true + }, + { + "TABLE 您a日.您a您a您a(test INT", + "TABLE `您a日`.`您a您a您a`(test INT", + true + }, + { + "TABLE 您a日.您a您a您a(Hi您Hi好Hi INT", + "TABLE `您a日`.`您a您a您a`(`Hi您Hi好Hi` INT", + true + }, + { + "--TABLE 您a日.您a您a您a(test INT", + "", + false + }, + { + "--您a日.您a您a您a(\n您Hi好", + "--您a日.您a您a您a(\n`您Hi好`", + true + }, + { + " /* TABLE 您a日.您a您a您a(test INT", + "", + false + }, + { + "/*您a日.您a您a您a(*/\n您Hi好", + "/*您a日.您a您a您a(*/\n`您Hi好`", + true + }, + { + " 您a日.您您aa您a /* 您a日.您a您a您a */ a您a日a.a您您您a", + " `您a日`.`您您aa您a` /* 您a日.您a您a您a */ `a您a日a`.`a您您您a`", + true + }, + //{ TODO + // "TABLE 您2.您a您a您a(test INT", + // "TABLE `您2`.`您a您a您a`(test INT", + // true + //}, + { + "skip 您a您a您a skip", + "skip `您a您a您a` skip", + true + }, + { + "test 您a2您3a您a 4 again", + "test `您a2您3a您a` 4 again", + true + }, + { + "CREATE TABLE db.`道渠`", + "", + false + }, + { + "CREATE TABLE db.`道渠", + "", + false + }, + { + "CREATE TABLE db.道渠", + "CREATE TABLE db.`道渠`", + true + }, + { + "CREATE TABLE db. 道渠", + "CREATE TABLE db. `道渠`", + true + }, + { + R"sql( + CREATE TABLE gb2312.`道渠` ( `id` int NOT NULL, + 您 INT, + 道渠 DATETIME, + 您test INT, test您 INT, test您test INT, + 道渠test INT, test道渠 INT, test道渠test INT, + 您_ INT, _您 INT, _您_ INT, + 您您__ INT, __您您 INT, __您您__ INT, + 您2 INT, 2您 INT, 2您2 INT, + 您您22 INT, 22您您 INT, 22您您22 INT, + 您_2 INT, _2您 INT, _2您_2 INT, _2您2_ INT, 2_您_2 INT, + 您您__22 INT, __22您您 INT, __22您您__22 INT, __22您您22__ INT, 22__您您__22 INT, + 您2_ INT, 2_您 INT, 2_您2_ INT, + 您您22__ INT, 22__您您 INT, 22__您您22__ INT, + 您_test INT, _test您 INT, _test您_test INT, _test您test_ INT, test_您test_ INT, test_您_test INT, + 您您_test INT, _test您您 INT, _test您您_test INT, _test您您test_ INT, test_您您test_ INT, test_您您_test INT, + 您test3 INT, test3您 INT, test3您test3 INT, test3您3test INT, + 您您test3 INT, test3您您 INT, test3您您test3 INT, test3您您3test INT, + 您3test INT, 3test您 INT, 3test您3test INT, 3test您test3 INT, + 您您3test INT, 3test您您 INT, 3test您您3test INT, 3test您您test3 INT, + 您_test4 INT, _test4您 INT, _test4您_test4 INT, test4_您_test4 INT, _test4您4test_ INT, _test4您test4_ INT, + 您您_test4 INT, _test4您您 INT, _test4您您_test4 INT, test4_您您_test4 INT, _test4您您4test_ INT, _test4您您test4_ INT, + 您_5test INT, _5test您 INT, _5test您_5test INT, 5test_您_test5 INT, _4test您test4_ INT, + test_日期 varchar(256), test_道_2 varchar(256) NOT NULL , + test_道渠您_3 + BIGINT NOT NULL, + 道您3_test INT, + PRIMARY KEY (`id`)) ENGINE=InnoDB DEFAULT CHARSET=gb2312; + )sql", + R"sql( + CREATE TABLE gb2312.`道渠` ( `id` int NOT NULL, + `您` INT, + `道渠` DATETIME, + `您test` INT, `test您` INT, `test您test` INT, + `道渠test` INT, `test道渠` INT, `test道渠test` INT, + `您_` INT, `_您` INT, `_您_` INT, + `您您__` INT, `__您您` INT, `__您您__` INT, + `您2` INT, `2您` INT, `2您2` INT, + `您您22` INT, `22您您` INT, `22您您22` INT, + `您_2` INT, `_2您` INT, `_2您_2` INT, `_2您2_` INT, `2_您_2` INT, + `您您__22` INT, `__22您您` INT, `__22您您__22` INT, `__22您您22__` INT, `22__您您__22` INT, + `您2_` INT, `2_您` INT, `2_您2_` INT, + `您您22__` INT, `22__您您` INT, `22__您您22__` INT, + `您_test` INT, `_test您` INT, `_test您_test` INT, `_test您test_` INT, `test_您test_` INT, `test_您_test` INT, + `您您_test` INT, `_test您您` INT, `_test您您_test` INT, `_test您您test_` INT, `test_您您test_` INT, `test_您您_test` INT, + `您test3` INT, `test3您` INT, `test3您test3` INT, `test3您3test` INT, + `您您test3` INT, `test3您您` INT, `test3您您test3` INT, `test3您您3test` INT, + `您3test` INT, `3test您` INT, `3test您3test` INT, `3test您test3` INT, + `您您3test` INT, `3test您您` INT, `3test您您3test` INT, `3test您您test3` INT, + `您_test4` INT, `_test4您` INT, `_test4您_test4` INT, `test4_您_test4` INT, `_test4您4test_` INT, `_test4您test4_` INT, + `您您_test4` INT, `_test4您您` INT, `_test4您您_test4` INT, `test4_您您_test4` INT, `_test4您您4test_` INT, `_test4您您test4_` INT, + `您_5test` INT, `_5test您` INT, `_5test您_5test` INT, `5test_您_test5` INT, `_4test您test4_` INT, + `test_日期` varchar(256), `test_道_2` varchar(256) NOT NULL , + `test_道渠您_3` + BIGINT NOT NULL, + `道您3_test` INT, + PRIMARY KEY (`id`)) ENGINE=InnoDB DEFAULT CHARSET=gb2312; + )sql", + true + }, +})); diff --git a/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp new file mode 100644 index 00000000000..cd4603ddaec --- /dev/null +++ b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp @@ -0,0 +1,96 @@ +#include +#include +#include + +namespace DB +{ + +/// Checks if there are no any tokens (like whitespaces) between current and previous pos +static bool noWhitespaces(const char * to, const char * from) +{ + return static_cast(from - to) == 0; +} + +/// Checks if the token should be quoted too together with unrecognized +static bool isWordOrNumber(TokenType type) +{ + return type == TokenType::BareWord || type == TokenType::Number; +} + +static void quoteLiteral( + IParser::Pos & pos, + IParser::Pos & pos_prev, + const char *& pos_unrecognized, + const char *& copy_from, + String & rewritten_query) +{ + /// Copy also whitespaces if any + const auto * end = + isWordOrNumber(pos->type) && noWhitespaces(pos_prev->end, pos->begin) + ? pos->end + : pos_prev->end; + String literal(pos_unrecognized, static_cast(end - pos_unrecognized)); + rewritten_query.append(copy_from, pos_unrecognized - copy_from).append(backQuoteMySQL(literal)); + copy_from = end; +} + +bool tryQuoteUnrecognizedTokens(const String & query, String & res) +{ + Tokens tokens(query.data(), query.data() + query.size()); + IParser::Pos pos(tokens, 0); + Expected expected; + String rewritten_query; + const char * copy_from = query.data(); + auto pos_prev = pos; + const char * pos_unrecognized = nullptr; + for (;pos->type != TokenType::EndOfStream; ++pos) + { + /// Commit quotes if any whitespaces found or the token is not a word + bool commit = !noWhitespaces(pos_prev->end, pos->begin) || (pos->type != TokenType::Error && !isWordOrNumber(pos->type)); + if (pos_unrecognized && commit) + { + quoteLiteral( + pos, + pos_prev, + pos_unrecognized, + copy_from, + rewritten_query); + pos_unrecognized = nullptr; + } + if (pos->type == TokenType::Error) + { + /// Find first appearance of the error token + if (!pos_unrecognized) + { + pos_unrecognized = + isWordOrNumber(pos_prev->type) && noWhitespaces(pos_prev->end, pos->begin) + ? pos_prev->begin + : pos->begin; + } + } + pos_prev = pos; + } + + /// There was EndOfStream but not committed unrecognized token + if (pos_unrecognized) + { + quoteLiteral( + pos, + pos_prev, + pos_unrecognized, + copy_from, + rewritten_query); + pos_unrecognized = nullptr; + } + + /// If no Errors found + if (copy_from == query.data()) + return false; + + auto size = static_cast(pos->end - copy_from); + rewritten_query.append(copy_from, size); + res = rewritten_query; + return true; +} + +} diff --git a/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h new file mode 100644 index 00000000000..582a297c485 --- /dev/null +++ b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +bool tryQuoteUnrecognizedTokens(const String & query, String & res); + +} diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3e928c3a811..b0a220eb1d2 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -34,16 +35,6 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -static String backQuoteMySQL(const String & x) -{ - String res(x.size(), '\0'); - { - WriteBufferFromString wb(res); - writeBackQuotedStringMySQL(x, wb); - } - return res; -} - StorageMySQL::StorageMySQL( const StorageID & table_id_, mysqlxx::PoolWithFailover && pool_, 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 c97c3e5e2a8..9130ccc359c 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1581,6 +1581,128 @@ def utf8mb4_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE utf8mb4_test") +def utf8mb4_column_test(clickhouse_node, mysql_node, service_name): + db = "utf8mb4_column_test" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + + # Full sync + mysql_node.query(f"CREATE TABLE {db}.unquoted (id INT primary key, 日期 DATETIME)") + mysql_node.query(f"CREATE TABLE {db}.quoted (id INT primary key, `日期` DATETIME)") + mysql_node.query(f"INSERT INTO {db}.unquoted VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.quoted VALUES(1, now())") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + + # Full sync replicated unquoted columns names since they use SHOW CREATE TABLE + # which returns quoted column names + check_query( + clickhouse_node, + f"/* expect: quoted unquoted */ SHOW TABLES FROM {db}", + "quoted\nunquoted\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.unquoted", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.quoted", + "1\n", + ) + + # Inc sync + mysql_node.query( + f"CREATE TABLE {db}.unquoted_new (id INT primary key, 日期 DATETIME)" + ) + mysql_node.query( + f"CREATE TABLE {db}.quoted_new (id INT primary key, `日期` DATETIME)" + ) + mysql_node.query(f"INSERT INTO {db}.unquoted_new VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.quoted_new VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.unquoted VALUES(2, now())") + mysql_node.query(f"INSERT INTO {db}.quoted VALUES(2, now())") + check_query( + clickhouse_node, + f"/* expect: 2 */ SELECT COUNT() FROM {db}.quoted", + "2\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.quoted_new", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 2 */ SELECT COUNT() FROM {db}.unquoted", + "2\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.unquoted_new", + "1\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + + +def utf8mb4_name_test(clickhouse_node, mysql_node, service_name): + db = "您Hi您" + table = "日期" + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"CREATE DATABASE `{db}`") + mysql_node.query( + f"CREATE TABLE `{db}`.`{table}` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO `{db}`.`{table}` VALUES(1, now())") + mysql_node.query( + f"CREATE TABLE {db}.{table}_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO {db}.{table}_unquoted VALUES(1, now())") + clickhouse_node.query( + f"CREATE DATABASE `{db}` ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}`", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}_unquoted`", + "1\n", + ) + + # Inc sync + mysql_node.query( + f"CREATE TABLE `{db}`.`{table}2` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO `{db}`.`{table}2` VALUES(1, now())") + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2`", + "1\n", + ) + + mysql_node.query( + f"CREATE TABLE {db}.{table}2_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO {db}.{table}2_unquoted VALUES(1, now())") + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2_unquoted`", + "1\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + + def system_parts_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS system_parts_test") clickhouse_node.query("DROP DATABASE IF EXISTS system_parts_test") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 32c1da8a2bd..e31ef70b4ad 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -381,6 +381,12 @@ def test_utf8mb4( ): materialized_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") materialized_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") + materialized_with_ddl.utf8mb4_column_test( + clickhouse_node, started_mysql_8_0, "mysql80" + ) + materialized_with_ddl.utf8mb4_name_test( + clickhouse_node, started_mysql_8_0, "mysql80" + ) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): From 3710c7238d9eaf0328170bafb03eb4b15ea5d67c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 09:19:06 +0000 Subject: [PATCH 699/871] Fix test_throttling --- tests/integration/test_throttling/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index ff8e7154d0d..2b5e9312a4c 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -114,7 +114,7 @@ def node_update_config(mode, setting, value=None): def assert_took(took, should_took): - assert took >= should_took[0] * 0.9 and took < should_took[1] + assert took >= should_took[0] * 0.85 and took < should_took[1] @pytest.mark.parametrize( From 2471b032ab7a504d1997e9d3681bf97f0564273d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 09:52:49 +0000 Subject: [PATCH 700/871] 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 d773f380377..06a9b62d9de 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 00000000000..6529ff889b0 --- /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 00000000000..fd446a8efe8 --- /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 5da6c99f6df90ae5a8dde59f9cccce8cee48fc61 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 12:02:27 +0200 Subject: [PATCH 701/871] Add comment --- tests/integration/test_throttling/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 2b5e9312a4c..62640394a85 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -114,6 +114,9 @@ def node_update_config(mode, setting, value=None): def assert_took(took, should_took): + # we need to decrease the lower limit because the server limits could + # be enforced by throttling some server background IO instead of query IO + # and we have no control over it assert took >= should_took[0] * 0.85 and took < should_took[1] From 79cc81890316338e35f13576cfd0360494e72645 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 11:06:21 +0000 Subject: [PATCH 702/871] 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 5fafb773d16..876766d0780 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 21097209d2e709db8022782a02980e52a7bc5df7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 24 Jul 2023 15:41:21 +0300 Subject: [PATCH 703/871] Revert "Remove `toDecimalString`" --- .../functions/type-conversion-functions.md | 38 +++ .../functions/type-conversion-functions.md | 38 +++ src/Functions/FunctionToDecimalString.cpp | 22 ++ src/Functions/FunctionToDecimalString.h | 312 ++++++++++++++++++ src/IO/WriteHelpers.h | 39 ++- .../02676_to_decimal_string.reference | 21 ++ .../0_stateless/02676_to_decimal_string.sql | 35 ++ 7 files changed, 492 insertions(+), 13 deletions(-) create mode 100644 src/Functions/FunctionToDecimalString.cpp create mode 100644 src/Functions/FunctionToDecimalString.h create mode 100644 tests/queries/0_stateless/02676_to_decimal_string.reference create mode 100644 tests/queries/0_stateless/02676_to_decimal_string.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index c2bd525c483..36f40b37238 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -945,6 +945,44 @@ Result: └────────────┴───────┘ ``` +## toDecimalString + +Converts a numeric value to String with the number of fractional digits in the output specified by the user. + +**Syntax** + +``` sql +toDecimalString(number, scale) +``` + +**Parameters** + +- `number` — Value to be represented as String, [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md), [Decimal](/docs/en/sql-reference/data-types/decimal.md), +- `scale` — Number of fractional digits, [UInt8](/docs/en/sql-reference/data-types/int-uint.md). + * Maximum scale for [Decimal](/docs/en/sql-reference/data-types/decimal.md) and [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md) types is 77 (it is the maximum possible number of significant digits for Decimal), + * Maximum scale for [Float](/docs/en/sql-reference/data-types/float.md) is 60. + +**Returned value** + +- Input value represented as [String](/docs/en/sql-reference/data-types/string.md) with given number of fractional digits (scale). + The number is rounded up or down according to common arithmetic in case requested scale is smaller than original number's scale. + +**Example** + +Query: + +``` sql +SELECT toDecimalString(CAST('64.32', 'Float64'), 5); +``` + +Result: + +```response +┌toDecimalString(CAST('64.32', 'Float64'), 5)─┐ +│ 64.32000 │ +└─────────────────────────────────────────────┘ +``` + ## reinterpretAsUInt(8\|16\|32\|64) ## reinterpretAsInt(8\|16\|32\|64) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 088b1a9a1f1..e53104d8d71 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -762,6 +762,44 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; └────────────┴───────┘ ``` +## toDecimalString + +Принимает любой численный тип первым аргументом, возвращает строковое десятичное представление числа с точностью, заданной вторым аргументом. + +**Синтаксис** + +``` sql +toDecimalString(number, scale) +``` + +**Параметры** + +- `number` — Значение любого числового типа: [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md), [Float](/docs/ru/sql-reference/data-types/float.md), [Decimal](/docs/ru/sql-reference/data-types/decimal.md), +- `scale` — Требуемое количество десятичных знаков после запятой, [UInt8](/docs/ru/sql-reference/data-types/int-uint.md). + * Значение `scale` для типов [Decimal](/docs/ru/sql-reference/data-types/decimal.md) и [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md) должно не превышать 77 (так как это наибольшее количество значимых символов для этих типов), + * Значение `scale` для типа [Float](/docs/ru/sql-reference/data-types/float.md) не должно превышать 60. + +**Возвращаемое значение** + +- Строка ([String](/docs/en/sql-reference/data-types/string.md)), представляющая собой десятичное представление входного числа с заданной длиной дробной части. + При необходимости число округляется по стандартным правилам арифметики. + +**Пример использования** + +Запрос: + +``` sql +SELECT toDecimalString(CAST('64.32', 'Float64'), 5); +``` + +Результат: + +```response +┌─toDecimalString(CAST('64.32', 'Float64'), 5)┐ +│ 64.32000 │ +└─────────────────────────────────────────────┘ +``` + ## reinterpretAsUInt(8\|16\|32\|64) {#reinterpretasuint8163264} ## reinterpretAsInt(8\|16\|32\|64) {#reinterpretasint8163264} diff --git a/src/Functions/FunctionToDecimalString.cpp b/src/Functions/FunctionToDecimalString.cpp new file mode 100644 index 00000000000..fe417b19137 --- /dev/null +++ b/src/Functions/FunctionToDecimalString.cpp @@ -0,0 +1,22 @@ +#include +#include +#include + +namespace DB +{ + +REGISTER_FUNCTION(ToDecimalString) +{ + factory.registerFunction( + FunctionDocumentation{ + .description=R"( +Returns string representation of a number. First argument is the number of any numeric type, +second argument is the desired number of digits in fractional part. Returns String. + + )", + .examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}}, + .categories{"String"} + }, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h new file mode 100644 index 00000000000..6ae007e6b66 --- /dev/null +++ b/src/Functions/FunctionToDecimalString.h @@ -0,0 +1,312 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; +} + +class FunctionToDecimalString : public IFunction +{ +public: + static constexpr auto name = "toDecimalString"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(*arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal first argument for formatDecimal function: got {}, expected numeric type", + arguments[0]->getName()); + + if (!isUInt8(*arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal second argument for formatDecimal function: got {}, expected UInt8", + arguments[1]->getName()); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + +private: + /// For operations with Integer/Float + template + void vectorConstant(const FromVectorType & vec_from, UInt8 precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, + /// and buffer resizes on-the-go. Also, .count() provided by buffer is convenient in this case. + WriteBufferFromVector buf_to(vec_to); + + for (size_t i = 0; i < input_rows_count; ++i) + { + format(vec_from[i], buf_to, precision); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + template + void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + format(vec_from[i], buf_to, vec_precision[i]); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + template + void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_precision.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + format(value_from, buf_to, vec_precision[i]); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + /// For operations with Decimal + template + void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. + constexpr size_t max_digits = std::numeric_limits::digits10; + if (precision > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + + WriteBufferFromVector buf_to(vec_to); + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + for (size_t i = 0; i < input_rows_count; ++i) + { + writeText(vec_from[i], from_scale, buf_to, true, true, precision); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + writeText(vec_from[i], from_scale, buf_to, true, true, vec_precision[i]); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + size_t input_rows_count = vec_precision.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + static void format(T value, DB::WriteBuffer & out, UInt8 precision) + { + /// Maximum of 60 is hard-coded in 'double-conversion/double-conversion.h' for floating point values, + /// Catch this here to give user a more reasonable error. + if (precision > 60) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too high precision requested for Float, must not be more than 60, got {}", Int8(precision)); + + DB::DoubleConverter::BufferType buffer; + double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; + + const auto result = DB::DoubleConverter::instance().ToFixed(value, precision, &builder); + + if (!result) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Error processing number: {}", value); + + out.write(buffer, builder.position()); + writeChar(0, out); + } + + template + static void format(T value, DB::WriteBuffer & out, UInt8 precision) + { + /// Fractional part for Integer is just trailing zeros. Let's limit it with 77 (like with Decimals). + constexpr size_t max_digits = std::numeric_limits::digits10; + if (precision > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + writeText(value, out); + if (precision > 0) [[likely]] + { + writeChar('.', out); + for (int i = 0; i < precision; ++i) + writeChar('0', out); + writeChar(0, out); + } + } + +public: + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + switch (arguments[0].type->getTypeId()) + { + case TypeIndex::UInt8: return executeType(arguments); + case TypeIndex::UInt16: return executeType(arguments); + case TypeIndex::UInt32: return executeType(arguments); + case TypeIndex::UInt64: return executeType(arguments); + case TypeIndex::UInt128: return executeType(arguments); + case TypeIndex::UInt256: return executeType(arguments); + case TypeIndex::Int8: return executeType(arguments); + case TypeIndex::Int16: return executeType(arguments); + case TypeIndex::Int32: return executeType(arguments); + case TypeIndex::Int64: return executeType(arguments); + case TypeIndex::Int128: return executeType(arguments); + case TypeIndex::Int256: return executeType(arguments); + case TypeIndex::Float32: return executeType(arguments); + case TypeIndex::Float64: return executeType(arguments); + case TypeIndex::Decimal32: return executeType(arguments); + case TypeIndex::Decimal64: return executeType(arguments); + case TypeIndex::Decimal128: return executeType(arguments); + case TypeIndex::Decimal256: return executeType(arguments); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments[0].column->getName(), getName()); + } + } + +private: + template + ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const + { + const auto * from_col_const = typeid_cast(arguments[0].column.get()); + const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); + const auto * precision_col_const = typeid_cast(arguments[1].column.get()); + + auto result_col = ColumnString::create(); + auto * result_col_string = assert_cast(result_col.get()); + ColumnString::Chars & result_chars = result_col_string->getChars(); + ColumnString::Offsets & result_offsets = result_col_string->getOffsets(); + + if constexpr (is_decimal) + { + const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); + UInt8 from_scale = from_col->getScale(); + + if (from_col) + { + if (precision_col_const) + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); + else + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); + } + else if (from_col_const) + constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); + } + else + { + const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); + if (from_col) + { + if (precision_col_const) + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); + else + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); + } + else if (from_col_const) + constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); + } + + return result_col; + } +}; + +} diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 76778543bd0..aa4c9b17e48 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -905,26 +905,26 @@ inline void writeText(const IPv4 & x, WriteBuffer & buf) { writeIPv4Text(x, buf) inline void writeText(const IPv6 & x, WriteBuffer & buf) { writeIPv6Text(x, buf); } template -void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) +void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, + bool fixed_fractional_length, UInt32 fractional_length) { /// If it's big integer, but the number of digits is small, /// use the implementation for smaller integers for more efficient arithmetic. - if constexpr (std::is_same_v) { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } } @@ -932,24 +932,36 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } } constexpr size_t max_digits = std::numeric_limits::digits10; assert(scale <= max_digits); + assert(fractional_length <= max_digits); + char buf[max_digits]; - memset(buf, '0', scale); + memset(buf, '0', std::max(scale, fractional_length)); T value = x; Int32 last_nonzero_pos = 0; - for (Int32 pos = scale - 1; pos >= 0; --pos) + + if (fixed_fractional_length && fractional_length < scale) + { + T new_value = value / DecimalUtils::scaleMultiplier(scale - fractional_length - 1); + auto round_carry = new_value % 10; + value = new_value / 10; + if (round_carry >= 5) + value += 1; + } + + for (Int32 pos = fixed_fractional_length ? std::min(scale - 1, fractional_length - 1) : scale - 1; pos >= 0; --pos) { auto remainder = value % 10; value /= 10; @@ -961,11 +973,12 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool } writeChar('.', ostr); - ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1); + ostr.write(buf, fixed_fractional_length ? fractional_length : (trailing_zeros ? scale : last_nonzero_pos + 1)); } template -void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) +void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, + bool fixed_fractional_length = false, UInt32 fractional_length = 0) { T part = DecimalUtils::getWholePart(x, scale); @@ -976,7 +989,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer writeIntText(part, ostr); - if (scale) + if (scale || (fixed_fractional_length && fractional_length > 0)) { part = DecimalUtils::getFractionalPart(x, scale); if (part || trailing_zeros) @@ -984,7 +997,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer if (part < 0) part *= T(-1); - writeDecimalFractional(part, scale, ostr, trailing_zeros); + writeDecimalFractional(part, scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); } } } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.reference b/tests/queries/0_stateless/02676_to_decimal_string.reference new file mode 100644 index 00000000000..4c27ee5b528 --- /dev/null +++ b/tests/queries/0_stateless/02676_to_decimal_string.reference @@ -0,0 +1,21 @@ +2.00000000000000000000000000000000000000000000000000000000000000000000000000000 +2.12 +-2.00000000000000000000000000000000000000000000000000000000000000000000000000000 +-2.12 +2.987600000000000033395508580724708735942840576171875000000000 +2.15 +-2.987600000000000033395508580724708735942840576171875000000000 +-2.15 +64.1230010986 +64.2340000000 +-64.1230010986 +-64.2340000000 +-32.345 +32.34500000000000000000000000000000000000000000000000000000000000000000000000000 +32.46 +-64.5671232345 +128.78932312332132985464 +-128.78932312332132985464 +128.78932312332132985464000000000000000000000000000000000000000000000000000000000 +128.7893231233 +-128.78932312332132985464123123789323123321329854600000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql new file mode 100644 index 00000000000..563d60c62c7 --- /dev/null +++ b/tests/queries/0_stateless/02676_to_decimal_string.sql @@ -0,0 +1,35 @@ +-- Regular types +SELECT toDecimalString(2, 77); -- more digits required than exist +SELECT toDecimalString(2.123456, 2); -- rounding +SELECT toDecimalString(-2, 77); -- more digits required than exist +SELECT toDecimalString(-2.123456, 2); -- rounding + +SELECT toDecimalString(2.9876, 60); -- more digits required than exist (took 60 as it is float by default) +SELECT toDecimalString(2.1456, 2); -- rounding +SELECT toDecimalString(-2.9876, 60); -- more digits required than exist +SELECT toDecimalString(-2.1456, 2); -- rounding + +-- Float32 and Float64 tests. No sense to test big float precision -- the result will be a mess anyway. +SELECT toDecimalString(64.123::Float32, 10); +SELECT toDecimalString(64.234::Float64, 10); +SELECT toDecimalString(-64.123::Float32, 10); +SELECT toDecimalString(-64.234::Float64, 10); + +-- Decimals +SELECT toDecimalString(-32.345::Decimal32(3), 3); +SELECT toDecimalString(32.345::Decimal32(3), 77); -- more digits required than exist +SELECT toDecimalString(32.456::Decimal32(3), 2); -- rounding +SELECT toDecimalString('-64.5671232345'::Decimal64(10), 10); +SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 20); +SELECT toDecimalString('-128.78932312332132985464123123'::Decimal128(26), 20); -- rounding +SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 77); -- more digits required than exist +SELECT toDecimalString('128.789323123321329854641231237893231233213298546'::Decimal256(45), 10); -- rounding +SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 77); -- more digits required than exist + +-- Max number of decimal fractional digits is defined as 77 for Int/UInt/Decimal and 60 for Float. +-- These values shall work OK. +SELECT toDecimalString('32.32'::Float32, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} From df5ff1383c5c6f7e24cb6933246fc04cf5dfe702 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 14:57:05 +0200 Subject: [PATCH 704/871] Fix settings not applied for explain query when format provided (#51859) --- src/Interpreters/InterpreterSetQuery.cpp | 3 +++ ..._explain_settings_not_applied_bug.reference | 11 +++++++++++ .../02798_explain_settings_not_applied_bug.sql | 18 ++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference create mode 100644 tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 6db57a4f950..e9118b747e5 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -65,6 +65,9 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta } else if (const auto * explain_query = ast->as()) { + if (explain_query->settings_ast) + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(); + applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } else if (const auto * query_with_output = dynamic_cast(ast.get())) diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference new file mode 100644 index 00000000000..6fc36a0ba01 --- /dev/null +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference @@ -0,0 +1,11 @@ + explain + + (Expression) + ExpressionTransform + (Aggregating) + FinalizeAggregatedTransform + AggregatingInOrderTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql new file mode 100644 index 00000000000..76f2129abfa --- /dev/null +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql @@ -0,0 +1,18 @@ +SET read_in_order_two_level_merge_threshold=1000000; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(a UInt64) +ENGINE = MergeTree +ORDER BY a; + +INSERT INTO t SELECT * FROM numbers_mt(1e3); +OPTIMIZE TABLE t FINAL; + +EXPLAIN PIPELINE +SELECT a +FROM t +GROUP BY a +FORMAT PrettySpace +SETTINGS optimize_aggregation_in_order = 1; + +DROP TABLE t; 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 705/871] 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 48adf36e678..3eba9a9de24 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 94fa2f18b174a0f88b59479aac40fbc2abbe73a7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 15:16:56 +0200 Subject: [PATCH 706/871] fix analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 17 +++++++++++++++++ .../02701_non_parametric_function.reference | 1 - .../02701_non_parametric_function.sql | 9 --------- 3 files changed, 17 insertions(+), 10 deletions(-) delete mode 100644 tests/queries/0_stateless/02701_non_parametric_function.reference delete mode 100644 tests/queries/0_stateless/02701_non_parametric_function.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index da8933aabaa..fd16b6e168b 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -116,6 +116,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first. @@ -4896,6 +4897,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi lambda_expression_untyped->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + if (!parameters.empty()) + { + throw Exception( + ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", function_node.formatASTForErrorMessage()); + } + auto lambda_expression_clone = lambda_expression_untyped->clone(); IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); @@ -5012,9 +5019,13 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); + bool is_executable_udf = true; if (!function) + { function = FunctionFactory::instance().tryGet(function_name, scope.context); + is_executable_udf = false; + } if (!function) { @@ -5065,6 +5076,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi return result_projection_names; } + /// Executable UDFs may have parameters. They are checked in UserDefinedExecutableFunctionFactory. + if (!parameters.empty() && !is_executable_udf) + { + throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", function_name); + } + /** For lambda arguments we need to initialize lambda argument types DataTypeFunction using `getLambdaArgumentTypes` function. * Then each lambda arguments are initialized with columns, where column source is lambda. * This information is important for later steps of query processing. diff --git a/tests/queries/0_stateless/02701_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference deleted file mode 100644 index 00750edc07d..00000000000 --- a/tests/queries/0_stateless/02701_non_parametric_function.reference +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql deleted file mode 100644 index 11c03372a73..00000000000 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ /dev/null @@ -1,9 +0,0 @@ --- Tags: no-parallel - -SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } - -CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); - -SELECT sum_udf(1)(1, 2); - -DROP FUNCTION IF EXISTS sum_udf; From c6e6fd761317662c05532d695c20be72f8e847d2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 15:58:21 +0200 Subject: [PATCH 707/871] Shard `OpenedFileCache` to avoid lock contention (#51341) * shard OpenedFileCache to avoid lock contention * Update OpenedFileCache.h * fix build --------- Co-authored-by: Alexey Milovidov --- src/Common/ProfileEvents.cpp | 1 + src/IO/OpenedFileCache.h | 109 +++++++++++++++++++++-------------- 2 files changed, 68 insertions(+), 42 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4a656e38edf..f18a67fa565 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -45,6 +45,7 @@ M(MMappedFileCacheMisses, "Number of times a file has not been found in the MMap cache (for the 'mmap' read_method), so we had to mmap it again.") \ M(OpenedFileCacheHits, "Number of times a file has been found in the opened file cache, so we didn't have to open it again.") \ M(OpenedFileCacheMisses, "Number of times a file has been found in the opened file cache, so we had to open it again.") \ + M(OpenedFileCacheMicroseconds, "Amount of time spent executing OpenedFileCache methods.") \ M(AIOWrite, "Number of writes with Linux or FreeBSD AIO interface") \ M(AIOWriteBytes, "Number of bytes written with Linux or FreeBSD AIO interface") \ M(AIORead, "Number of reads with Linux or FreeBSD AIO interface") \ diff --git a/src/IO/OpenedFileCache.h b/src/IO/OpenedFileCache.h index 61e502a494b..2cecc675af7 100644 --- a/src/IO/OpenedFileCache.h +++ b/src/IO/OpenedFileCache.h @@ -4,14 +4,18 @@ #include #include -#include #include +#include +#include + +#include namespace ProfileEvents { extern const Event OpenedFileCacheHits; extern const Event OpenedFileCacheMisses; + extern const Event OpenedFileCacheMicroseconds; } namespace DB @@ -26,57 +30,79 @@ namespace DB */ class OpenedFileCache { -private: - using Key = std::pair; + class OpenedFileMap + { + using Key = std::pair; - using OpenedFileWeakPtr = std::weak_ptr; - using Files = std::map; + using OpenedFileWeakPtr = std::weak_ptr; + using Files = std::map; - Files files; - std::mutex mutex; + Files files; + std::mutex mutex; + + public: + using OpenedFilePtr = std::shared_ptr; + + OpenedFilePtr get(const std::string & path, int flags) + { + Key key(path, flags); + + std::lock_guard lock(mutex); + + auto [it, inserted] = files.emplace(key, OpenedFilePtr{}); + if (!inserted) + { + if (auto res = it->second.lock()) + { + ProfileEvents::increment(ProfileEvents::OpenedFileCacheHits); + return res; + } + } + ProfileEvents::increment(ProfileEvents::OpenedFileCacheMisses); + + OpenedFilePtr res + { + new OpenedFile(path, flags), + [key, this](auto ptr) + { + { + std::lock_guard another_lock(mutex); + files.erase(key); + } + delete ptr; + } + }; + + it->second = res; + return res; + } + + void remove(const std::string & path, int flags) + { + Key key(path, flags); + std::lock_guard lock(mutex); + files.erase(key); + } + }; + + static constexpr size_t buckets = 1024; + std::vector impls{buckets}; public: - using OpenedFilePtr = std::shared_ptr; + using OpenedFilePtr = OpenedFileMap::OpenedFilePtr; OpenedFilePtr get(const std::string & path, int flags) { - Key key(path, flags); - - std::lock_guard lock(mutex); - - auto [it, inserted] = files.emplace(key, OpenedFilePtr{}); - if (!inserted) - { - if (auto res = it->second.lock()) - { - ProfileEvents::increment(ProfileEvents::OpenedFileCacheHits); - return res; - } - } - ProfileEvents::increment(ProfileEvents::OpenedFileCacheMisses); - - OpenedFilePtr res - { - new OpenedFile(path, flags), - [key, this](auto ptr) - { - { - std::lock_guard another_lock(mutex); - files.erase(key); - } - delete ptr; - } - }; - - it->second = res; - return res; + ProfileEventTimeIncrement watch(ProfileEvents::OpenedFileCacheMicroseconds); + const auto bucket = CityHash_v1_0_2::CityHash64(path.data(), path.length()) % buckets; + return impls[bucket].get(path, flags); } void remove(const std::string & path, int flags) { - Key key(path, flags); - std::lock_guard lock(mutex); - files.erase(key); + ProfileEventTimeIncrement watch(ProfileEvents::OpenedFileCacheMicroseconds); + const auto bucket = CityHash_v1_0_2::CityHash64(path.data(), path.length()) % buckets; + impls[bucket].remove(path, flags); } static OpenedFileCache & instance() @@ -87,5 +113,4 @@ public: }; using OpenedFileCachePtr = std::shared_ptr; - } 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 708/871] 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 f523e7b7cf9..1508d834592 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 709/871] 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 b746d1610a4..1d56b2c3a71 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 b9a7d17e955..d175d31846b 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 184e6321988..aee714a494e 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 7b03f3277e5097000bda03c93c27efc5d8c29a34 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 17:59:50 +0200 Subject: [PATCH 710/871] Try fix session on close test --- src/Coordination/KeeperDispatcher.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 02308f245d5..99c28674273 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -599,6 +599,10 @@ void KeeperDispatcher::sessionCleanerTask() void KeeperDispatcher::finishSession(int64_t session_id) { + /// shutdown() method will cleanup sessions if needed + if (shutdown_called) + return; + { std::lock_guard lock(session_to_response_callback_mutex); auto session_it = session_to_response_callback.find(session_id); From da0a332d46a7939377dca08147897d0f5b1dd9c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 18:09:38 +0200 Subject: [PATCH 711/871] Keep symtab --- cmake/split_debug_symbols.cmake | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index a9c2158359a..8ba7669d9e2 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -22,8 +22,9 @@ macro(clickhouse_split_debug_symbols) # Splits debug symbols into separate file, leaves the binary untouched: COMMAND "${OBJCOPY_PATH}" --only-keep-debug "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" - # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check: - COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" + # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check. + # Also, after we disabled the export of symbols for dynamic linking, we still to keep a static symbol table for good stack traces. + COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash --keep-section=.symtab --keep-section=.strtab --keep-section=.shstrtab "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" # Associate stripped binary with debug symbols: COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" COMMENT "Stripping clickhouse binary" VERBATIM From 0bbf26549f4fb49c599b4a58475c71bccfe9b37b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 18:13:15 +0200 Subject: [PATCH 712/871] 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 8d92d784226..9f595800bea 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 c755fde3a41f95466db1b5b9aeeb54decafa97c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 18:36:00 +0000 Subject: [PATCH 713/871] Include query cache usage in SYSTEM.QUERY_LOG --- docs/en/operations/query-cache.md | 6 +- docs/en/operations/system-tables/query_log.md | 6 ++ src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/Cache/QueryCache.h | 9 +++ src/Interpreters/QueryLog.cpp | 13 ++++ src/Interpreters/QueryLog.h | 5 +- src/Interpreters/executeQuery.cpp | 22 ++++--- src/Interpreters/executeQuery.h | 1 + .../02494_query_cache_query_log.reference | 16 +++++ .../02494_query_cache_query_log.sql | 64 +++++++++++++++++++ 10 files changed, 132 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_query_log.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_query_log.sql diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index bfa51650cd8..547105c65cc 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -62,8 +62,10 @@ may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table `system.query_cache`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table -`system.events`. Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = true". Other queries do not -affect the cache miss counter. +[system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = +true". Other queries do not affect the cache miss counter. Field `query_log_usage` in system table +[system.query_log](system-tables/query_log.md) shows for each ran query whether the query result was written into or read from the query +cache. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be changed (see below) but doing so is not recommended for security reasons. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index b9fdd19c643..b8dc0c0224c 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -111,6 +111,11 @@ Columns: - `used_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `functions`, which were used during query execution. - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. +- `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: + - `'None' = 1` = The query result was neither written into nor read from the query cache. + - `'Write' = 1` = The query result was written into the query cache. + - `'Read' = 1` = The query result was read from the query cache. + - `'Unknown' = 1` = Unknown status. **Example** @@ -186,6 +191,7 @@ used_formats: [] used_functions: [] used_storages: [] used_table_functions: [] +query_cache_usage: None ``` **See Also** diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 8d0f18cc305..c6596f50eda 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -605,7 +605,7 @@ try total_rows, total_bytes, key.query_str); bool pulling_pipeline = false; - logQueryFinish(query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, internal); + logQueryFinish(query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); } catch (...) { diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c24b09c8e46..973015b8003 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -24,6 +24,15 @@ bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context); class QueryCache { public: + enum class Usage + { + /// starts at 1 for compatibitity with DataTypeEnum8 + None = 1, /// query result neither written nor read into/from query cache + Write, /// query result wrote into query cache + Read, /// query result read from query cache + Unknown, /// we don't know what what happened + }; + /// Represents a query result in the cache. struct Key { diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index ec0315c2f95..c3294512f14 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -41,6 +41,15 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)} }); + auto query_cache_usage_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"None", static_cast(QueryCache::Usage::None)}, + {"Write", static_cast(QueryCache::Usage::Write)}, + {"Read", static_cast(QueryCache::Usage::Read)}, + {"Unknown", static_cast(QueryCache::Usage::Unknown)} + }); + auto low_cardinality_string = std::make_shared(std::make_shared()); auto array_low_cardinality_string = std::make_shared(low_cardinality_string); @@ -126,6 +135,8 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"transaction_id", getTransactionIDDataType()}, + {"query_cache_usage", std::move(query_cache_usage_datatype)}, + {"asynchronous_read_counters", std::make_shared(low_cardinality_string, std::make_shared())}, }; } @@ -277,6 +288,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); + columns[i++]->insert(query_cache_usage); + if (async_read_counters) async_read_counters->dumpToMapColumn(columns[i++].get()); else diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 570d1297239..5bc80280eac 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -4,8 +4,9 @@ #include #include #include -#include +#include #include +#include #include #include #include @@ -96,6 +97,8 @@ struct QueryLogElement TransactionID tid; + QueryCache::Usage query_cache_usage = QueryCache::Usage::Unknown; + static std::string name() { return "QueryLog"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 688d3b9967d..578ca3b41f9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -209,7 +209,7 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er } static void -addStatusInfoToQueryElement(QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) +addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) { const auto time_now = std::chrono::system_clock::now(); UInt64 elapsed_microseconds = info.elapsed_microseconds; @@ -347,6 +347,7 @@ void logQueryFinish( const QueryPipeline & query_pipeline, bool pulling_pipeline, std::shared_ptr query_span, + QueryCache::Usage query_cache_usage, bool internal) { const Settings & settings = context->getSettingsRef(); @@ -364,7 +365,7 @@ void logQueryFinish( QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events); elem.type = QueryLogElementType::QUERY_FINISH; - addStatusInfoToQueryElement(elem, info, query_ast, context); + addStatusInfoToQueryLogElement(elem, info, query_ast, context); if (pulling_pipeline) { @@ -399,6 +400,8 @@ void logQueryFinish( ReadableSize(elem.read_bytes / elapsed_seconds)); } + elem.query_cache_usage = query_cache_usage; + if (log_queries && elem.type >= log_queries_min_type && static_cast(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { @@ -499,13 +502,15 @@ void logQueryException( if (process_list_elem) { QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false); - addStatusInfoToQueryElement(elem, info, query_ast, context); + addStatusInfoToQueryLogElement(elem, info, query_ast, context); } else { elem.query_duration_ms = start_watch.elapsedMilliseconds(); } + elem.query_cache_usage = QueryCache::Usage::None; + if (settings.calculate_text_stack_trace && log_error) setExceptionStackTrace(elem); logException(context, elem, log_error); @@ -975,7 +980,7 @@ static std::tuple executeQueryImpl( QueryCachePtr query_cache = context->getQueryCache(); const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); - bool write_into_query_cache = false; + QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) { @@ -992,6 +997,7 @@ static std::tuple executeQueryImpl( QueryPipeline pipeline; pipeline.readFromQueryCache(reader.getSource(), reader.getSourceTotals(), reader.getSourceExtremes()); res.pipeline = std::move(pipeline); + query_cache_usage = QueryCache::Usage::Read; return true; } } @@ -1095,7 +1101,7 @@ static std::tuple executeQueryImpl( settings.query_cache_max_size_in_bytes, settings.query_cache_max_entries)); res.pipeline.writeResultIntoQueryCache(query_cache_writer); - write_into_query_cache = true; + query_cache_usage = QueryCache::Usage::Write; } } @@ -1147,19 +1153,19 @@ static std::tuple executeQueryImpl( auto finish_callback = [elem, context, ast, - write_into_query_cache, + query_cache_usage, internal, implicit_txn_control, execute_implicit_tcl_query, pulling_pipeline = pipeline.pulling(), query_span](QueryPipeline & query_pipeline) mutable { - if (write_into_query_cache) + if (query_cache_usage == QueryCache::Usage::Write) /// Trigger the actual write of the buffered query result into the query cache. This is done explicitly to prevent /// partial/garbage results in case of exceptions during query execution. query_pipeline.finalizeWriteInQueryCache(); - logQueryFinish(elem, context, ast, query_pipeline, pulling_pipeline, query_span, internal); + logQueryFinish(elem, context, ast, query_pipeline, pulling_pipeline, query_span, query_cache_usage, internal); if (*implicit_txn_control) execute_implicit_tcl_query(context, ASTTransactionControl::COMMIT); diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 53624f8c812..f2a12bbef18 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -92,6 +92,7 @@ void logQueryFinish( const QueryPipeline & query_pipeline, bool pulling_pipeline, std::shared_ptr query_span, + QueryCache::Usage query_cache_usage, bool internal); void logQueryException( diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.reference b/tests/queries/0_stateless/02494_query_cache_query_log.reference new file mode 100644 index 00000000000..9037909d121 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_query_log.reference @@ -0,0 +1,16 @@ +-- Run a query with query cache not enabled +124437993 +QueryStart SELECT 124437993; Unknown +QueryFinish SELECT 124437993; None +-- Run a query with query cache enabled +124437994 +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +-- Run the same query with query cache enabled +124437994 +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +-- Throw exception with query cache enabled +SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; None diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.sql b/tests/queries/0_stateless/02494_query_cache_query_log.sql new file mode 100644 index 00000000000..79a8f4cb62b --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_query_log.sql @@ -0,0 +1,64 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SYSTEM DROP QUERY CACHE; + +-- DROP TABLE system.query_log; -- debugging + + + +SELECT '-- Run a query with query cache not enabled'; +SELECT 124437993; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'None' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437993;' +ORDER BY type; + + + +SELECT '-- Run a query with query cache enabled'; +SELECT 124437994 SETTINGS use_query_cache = 1; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'Write' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' +ORDER BY type; + + + +SELECT '-- Run the same query with query cache enabled'; +SELECT 124437994 SETTINGS use_query_cache = 1; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'Read' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' +ORDER BY type; + + + +SELECT '-- Throw exception with query cache enabled'; +SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'None' +SELECT query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1;' + AND type = 'ExceptionWhileProcessing'; + +SYSTEM DROP QUERY CACHE; From 032956dd1eeca994d6fa5a66f974cfa10203c205 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 18:42:02 +0200 Subject: [PATCH 714/871] 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 2393f45ebb6..e11913fc3d2 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 a2a4ec92cf7..20b6a6c977f 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 test_shutdown_and_wait(start_cluster): 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 cc63af3676b..59899e1c14a 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 715/871] 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 aaaeeb7339d..f28194781c2 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 c3fbc08c4a9..2381def9151 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 716/871] 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 79168d82c54..e03701327b1 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 717/871] 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 f4505147ced..eba8109253d 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: 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 @@ private: 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 718/871] 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 180610288aa..17ff367a58d 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 719/871] 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 2ff8f161e99..33fa7955e0d 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 720/871] 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 00000000000..3adbbb3029e --- /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 721/871] 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 4336de41b7b..af47b6ff4cd 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 0037ab85c07..1b177b84afa 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 722/871] 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 9a5ef4727c5..8470b4a3379 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 723/871] 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 6cdcffab50a..e6431927805 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 1c721e3724b..daa39536fa7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -481,6 +481,9 @@ private: 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 724/871] 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 1ed6c83af7d..1489db55857 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 01910c4caff..922607da813 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -110,7 +110,7 @@ public: /// 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 0bcd62d30c7..8ea3f4a0aa5 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 e3e4a3f523c..d97558c4728 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 92c4edbac2a..f36c4154684 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 94fc07bcc4a..79929c4e66e 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 cbeb84ef2e7..a6d8763bfb8 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 6b068b0d8b1..cf283d3358c 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 0b7352e9cbb..b12ffc555d4 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 d6873d9490e..ff8ff2d2651 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 21c80b742fd..99bf1a7cc33 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 1a367a8199d..fd825720ac9 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 05b34e8460f..29096a38be6 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 193bb5b6ab0..92e6bcb326c 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 75d43b541e1..ae79b3f932e 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 6257e617d4a..2ef4f4d6218 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 9796ae10c07..72e25cc3cf9 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.h +++ b/src/Parsers/Kusto/ParserKQLOperators.h @@ -31,10 +31,10 @@ protected: 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 @@ protected: 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 d77ae8d3a27..18e91c533e0 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 b7adaa35335..676ce50d04f 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -35,7 +35,7 @@ public: /// - 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 e5f52936021..b1b08cdf256 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 afdff44020f..b132d27670d 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 7f7f9058f1b..1b20778877d 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 069670c84a5..29b75fa6552 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 cbd32460f7e..3126d584964 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 14b7fc15af2..582dc6f882d 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,7 +149,7 @@ public: 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 4e053c4598c..c3dedd69d0d 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 76108f1cdd4..ce3daff0785 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 0b3b86b4772..c28ca1cfc8a 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 ab55c294bfa8c04aa94494a1bdff3e3e3b339821 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 25 Jul 2023 07:25:27 +0000 Subject: [PATCH 725/871] Add assert for STANDARD storage class --- tests/integration/test_backup_s3_storage_class/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py index a4c93d40f0c..3c0a0cf7706 100644 --- a/tests/integration/test_backup_s3_storage_class/test.py +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -48,4 +48,6 @@ def test_backup_s3_storage_class(started_cluster): """ ) - assert True + minio = cluster.minio_client + lst = list(minio.list_objects(cluster.minio_bucket, "data/.backup")) + assert lst[0].storage_class == "STANDARD" From 3e3adc7fecd5f6c409320727bec3a0291aa2430b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:29:59 +0200 Subject: [PATCH 726/871] tests: increase throttling for 01923_network_receive_time_metric_insert In debug builds launching the client can take a while, so let's increase the throttling to avoid flakiness CI: https://s3.amazonaws.com/clickhouse-test-reports/52490/9e2526a5f04861fcfac49c2ce85560d08c68af66/stateless_tests__debug__[1_5].html Signed-off-by: Azat Khuzhin --- .../0_stateless/01923_network_receive_time_metric_insert.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index ec5aa141859..4d7e79fae52 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" # Rate limit is chosen for operation to spent more than one second. -seq 1 1000 | pv --quiet --rate-limit 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" +seq 1 1000 | pv --quiet --rate-limit 500 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" # We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client. ${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; From b02e290d5507419e6166433b0a045eaeb3d124d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:37:02 +0200 Subject: [PATCH 727/871] 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 | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.sh b/tests/queries/0_stateless/01035_avg_weighted_long.sh index 138aa03fbb3..8838b07a3d7 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") - -for left in "${types[@]}" -do - for right 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))" - done -done - exttypes=("Int128" "Int256" "UInt256") - -for left in "${exttypes[@]}" -do - for right in "${exttypes[@]}" - do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(to${left}(1), to${right}(2))" - done -done - # Decimal types dtypes=("32" "64" "128" "256") -for left in "${dtypes[@]}" -do - for right in "${dtypes[@]}" +( + for left in "${types[@]}" do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" + 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 + + for left in "${exttypes[@]}" + do + for right in "${exttypes[@]}" + do + echo "SELECT avgWeighted(to${left}(1), to${right}(2));" + done + done + + for left in "${dtypes[@]}" + do + for right in "${dtypes[@]}" + do + echo "SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4));" + 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 728/871] 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 fdc994aec33..57a7e228341 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 d500e75569c59d1f91ae3de9c43f24f2be703e21 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 12:07:47 +0200 Subject: [PATCH 729/871] fix --- src/Functions/FunctionToDecimalString.h | 67 ++----------------- .../0_stateless/02676_to_decimal_string.sql | 6 ++ 2 files changed, 13 insertions(+), 60 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index 6ae007e6b66..68ad978632e 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionToDecimalString : public IFunction @@ -36,17 +37,14 @@ public: size_t getNumberOfArguments() const override { return 2; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isNumber(*arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal first argument for formatDecimal function: got {}, expected numeric type", - arguments[0]->getName()); + FunctionArgumentDescriptors mandatory_args = { + {"Value", nullptr, nullptr, nullptr}, + {"precision", &isNativeInteger, &isColumnConst, "const Integer [0-77]"} + }; - if (!isUInt8(*arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal second argument for formatDecimal function: got {}, expected UInt8", - arguments[1]->getName()); + validateFunctionArgumentTypes(*this, arguments, mandatory_args, {}); return std::make_shared(); } @@ -98,29 +96,6 @@ private: buf_to.finalize(); } - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(value_from, buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - /// For operations with Decimal template void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, @@ -168,29 +143,6 @@ private: buf_to.finalize(); } - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - template static void format(T value, DB::WriteBuffer & out, UInt8 precision) { @@ -263,7 +215,6 @@ private: template ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const { - const auto * from_col_const = typeid_cast(arguments[0].column.get()); const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); const auto * precision_col_const = typeid_cast(arguments[1].column.get()); @@ -284,8 +235,6 @@ private: else vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); } @@ -299,8 +248,6 @@ private: else vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql index 563d60c62c7..1dae139deb1 100644 --- a/tests/queries/0_stateless/02676_to_decimal_string.sql +++ b/tests/queries/0_stateless/02676_to_decimal_string.sql @@ -33,3 +33,9 @@ SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} + +-- wrong types: #52407 and similar +SELECT toDecimalString('256.256'::Decimal256(45), *); -- {serverError ILLEGAL_COLUMN} +SELECT toDecimalString('128.128'::Decimal128(30), 'str'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT toDecimalString('64.64'::Decimal64(10)); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT toDecimalString('64.64'::Decimal64(10), 3, 3); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From 8184a289e5441208110bcd2f8f63b57e31ccde33 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 May 2023 01:53:20 +0000 Subject: [PATCH 730/871] Partially reimplement Parquet encoder to make it faster and parallelizable --- src/CMakeLists.txt | 4 + src/Common/CurrentMetrics.cpp | 6 +- src/Common/PODArray.cpp | 10 + src/Common/PODArray.h | 11 + src/Core/Settings.h | 4 + src/Formats/FormatFactory.cpp | 7 +- src/Formats/FormatSettings.h | 6 + .../Formats/Impl/CHColumnToArrowColumn.cpp | 5 +- .../Formats/Impl/Parquet/PrepareForWrite.cpp | 618 +++++++++++++ .../Formats/Impl/Parquet/ThriftUtil.cpp | 35 + .../Formats/Impl/Parquet/ThriftUtil.h | 17 + src/Processors/Formats/Impl/Parquet/Write.cpp | 816 ++++++++++++++++++ src/Processors/Formats/Impl/Parquet/Write.h | 135 +++ .../Formats/Impl/ParquetBlockOutputFormat.cpp | 467 +++++++++- .../Formats/Impl/ParquetBlockOutputFormat.h | 116 ++- .../02735_parquet_encoder.reference | 55 ++ .../0_stateless/02735_parquet_encoder.sql | 168 ++++ 17 files changed, 2425 insertions(+), 55 deletions(-) create mode 100644 src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ThriftUtil.h create mode 100644 src/Processors/Formats/Impl/Parquet/Write.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/Write.h create mode 100644 tests/queries/0_stateless/02735_parquet_encoder.reference create mode 100644 tests/queries/0_stateless/02735_parquet_encoder.sql diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 975bf9bb618..5c66c7e9495 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -267,6 +267,10 @@ add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations) add_object_library(clickhouse_user_defined_functions Functions/UserDefined) +if (USE_PARQUET) + add_object_library(clickhouse_processors_formats_impl_parquet Processors/Formats/Impl/Parquet) +endif() + if (TARGET ch_contrib::nuraft) add_object_library(clickhouse_coordination Coordination) endif() diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 583b13cf79d..9a4ffb0577a 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -149,8 +149,10 @@ M(RestartReplicaThreadsActive, "Number of threads in the RESTART REPLICA thread pool running a task.") \ M(QueryPipelineExecutorThreads, "Number of threads in the PipelineExecutor thread pool.") \ M(QueryPipelineExecutorThreadsActive, "Number of threads in the PipelineExecutor thread pool running a task.") \ - M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ - M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \ + M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool.") \ + M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ + M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ + M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ diff --git a/src/Common/PODArray.cpp b/src/Common/PODArray.cpp index 07c3cf1af1a..d21dc40867d 100644 --- a/src/Common/PODArray.cpp +++ b/src/Common/PODArray.cpp @@ -15,4 +15,14 @@ template class PODArray, PADDING_FOR_SIMD - 1, PADD template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; + +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; + +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; } diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b126afd2a37..68c1e325f0c 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -783,4 +783,15 @@ extern template class PODArray, PADDING_FOR_SIMD - extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; + +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; + +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; + } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 97c64ba133c..98f7f212aa5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -953,6 +953,10 @@ class IColumn; M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "lz4", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ + M(Bool, output_format_parquet_use_custom_encoder, true, "Use experimental faster Parquet encoder implementation.", 0) \ + M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \ + M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \ + M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6e3e086859b..663b7f1ba95 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -130,6 +130,10 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; + format_settings.parquet.use_custom_encoder = settings.output_format_parquet_use_custom_encoder; + format_settings.parquet.parallel_encoding = settings.output_format_parquet_parallel_encoding; + format_settings.parquet.data_page_size = settings.output_format_parquet_data_page_size; + format_settings.parquet.write_batch_size = settings.output_format_parquet_batch_size; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -434,7 +438,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( return format; } - return getOutputFormat(name, buf, sample, context, _format_settings); + return getOutputFormat(name, buf, sample, context, format_settings); } @@ -453,6 +457,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + format_settings.max_threads = context->getSettingsRef().max_threads; /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e321e5264ca..3259c46e5ff 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -100,6 +100,8 @@ struct FormatSettings UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; + size_t max_threads = 1; + enum class ArrowCompression { NONE, @@ -233,10 +235,14 @@ struct FormatSettings bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; bool preserve_order = false; + bool use_custom_encoder = true; + bool parallel_encoding = true; UInt64 max_block_size = 8192; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; bool output_compliant_nested_types = true; + size_t data_page_size = 1024 * 1024; + size_t write_batch_size = 1024; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index f688efa3290..e2383d1bfab 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -684,9 +684,6 @@ namespace DB bool output_fixed_string_as_fixed_byte_array, std::unordered_map & dictionary_values) { - const String column_type_name = column_type->getFamilyName(); - WhichDataType which(column_type); - switch (column_type->getTypeId()) { case TypeIndex::Nullable: @@ -796,7 +793,7 @@ namespace DB FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) #undef DISPATCH default: - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of a column '{}' is not supported for conversion into {} data format.", column_type_name, column_name, format_name); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of a column '{}' is not supported for conversion into {} data format.", column_type->getFamilyName(), column_name, format_name); } } diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp new file mode 100644 index 00000000000..a70b6fcfc81 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -0,0 +1,618 @@ +#include "Processors/Formats/Impl/Parquet/Write.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +/// This file deals with schema conversion and with repetition and definition levels. + +/// Schema conversion is pretty straightforward. + +/// "Repetition and definition levels" are a somewhat tricky way of encoding information about +/// optional fields and lists. +/// +/// If you don't want to learn how these work, feel free to skip the updateRepDefLevels* functions. +/// All you need to know is: +/// * values for nulls are not encoded, so we have to filter nullable columns, +/// * information about all array lengths and nulls is encoded in the arrays `def` and `rep`, +/// which need to be encoded next to the data, +/// * `def` and `rep` arrays can be longer than `primitive_column`, because they include nulls and +/// empty arrays; the values in primitive_column correspond to positions where def[i] == max_def. +/// +/// If you do want to learn it, dremel paper: https://research.google/pubs/pub36632/ +/// Instead of reading the whole paper, try staring at figures 2-3 for a while - it might be enough. +/// (Why does Parquet do all this instead of just storing array lengths and null masks? I'm not +/// really sure.) +/// +/// We calculate the levels recursively, from inner to outer columns. +/// This means scanning the whole array for each Array/Nullable nesting level, which is probably not +/// the most efficient way to do it. But there's usually at most one nesting level, so it's fine. +/// +/// Most of this is moot because ClickHouse doesn't support nullable arrays or tuples right now, so +/// almost none of the tricky cases can happen. We implement it in full generality anyway (mostly +/// because I only learned the previous sentence after writing most of the code). + + +namespace DB::ErrorCodes +{ + extern const int UNKNOWN_TYPE; + extern const int TOO_DEEP_RECURSION; // I'm 14 and this is deep + extern const int UNKNOWN_COMPRESSION_METHOD; + extern const int LOGICAL_ERROR; +} + +namespace DB::Parquet +{ + +/// Thrift structs that Parquet uses for various metadata inside the parquet file. +namespace parq = parquet::format; + +namespace +{ + +void assertNoDefOverflow(ColumnChunkWriteState & s) +{ + if (s.max_def == UINT8_MAX) + throw Exception(ErrorCodes::TOO_DEEP_RECURSION, + "Column has more than 255 levels of nested Array/Nullable. Impressive! Unfortunately, " + "this is not supported by this Parquet encoder (but is supported by Parquet, if you " + "really need this for some reason)."); +} + +void updateRepDefLevelsAndFilterColumnForNullable(ColumnChunkWriteState & s, const NullMap & null_map) +{ + /// Increment definition levels for non-nulls. + /// Filter the column to contain only non-null values. + + assertNoDefOverflow(s); + ++s.max_def; + + /// Normal case: no arrays or nullables inside this nullable. + if (s.max_def == 1) + { + chassert(s.def.empty()); + s.def.resize(null_map.size()); + for (size_t i = 0; i < s.def.size(); ++i) + s.def[i] = !null_map[i]; + + /// We could be more efficient with this: + /// * Instead of doing the filter() here, we could defer it to writeColumnChunkBody(), at + /// least in the simple case of Nullable(Primitive). Then it'll parallelize if the table + /// consists of one big tuple. + /// * Instead of filtering explicitly, we could build filtering into the data encoder. + /// * Instead of filling out the `def` values above, we could point to null_map and build + /// the '!' into the encoder. + /// None of these seem worth the complexity right now. + s.primitive_column = s.primitive_column->filter(s.def, /*result_size_hint*/ -1); + + return; + } + + /// Weird general case: Nullable(Array), Nullable(Nullable), or any arbitrary nesting like that. + /// This is currently not allowed in ClickHouse, but let's support it anyway just in case. + + IColumn::Filter filter; + size_t row_idx = static_cast(-1); + for (size_t i = 0; i < s.def.size(); ++i) + { + row_idx += s.max_rep == 0 || s.rep[i] == 0; + if (s.def[i] == s.max_def - 1) + filter.push_back(!null_map[row_idx]); + s.def[i] += !null_map[row_idx]; + } + s.primitive_column = s.primitive_column->filter(filter, /*result_size_hint*/ -1); +} + +void updateRepDefLevelsForArray(ColumnChunkWriteState & s, const IColumn::Offsets & offsets) +{ + /// Increment all definition levels. + /// For non-first elements of arrays, increment repetition levels. + /// For empty arrays, insert a zero into repetition and definition levels arrays. + + assertNoDefOverflow(s); + ++s.max_def; + ++s.max_rep; + + /// Common case: no arrays or nullables inside this array. + if (s.max_rep == 1 && s.max_def == 1) + { + s.def.resize_fill(s.primitive_column->size(), 1); + s.rep.resize_fill(s.primitive_column->size(), 1); + size_t i = 0; + for (ssize_t row = 0; row < static_cast(offsets.size()); ++row) + { + size_t n = offsets[row] - offsets[row - 1]; + if (n) + { + s.rep[i] = 0; + i += n; + } + else + { + s.def.push_back(1); + s.rep.push_back(1); + s.def[i] = 0; + s.rep[i] = 0; + i += 1; + } + } + return; + } + + /// General case: Array(Array), Array(Nullable), or any arbitrary nesting like that. + + for (auto & x : s.def) + ++x; + + if (s.max_rep == 1) + s.rep.resize_fill(s.def.size(), 1); + else + for (auto & x : s.rep) + ++x; + + PaddedPODArray mask(s.def.size(), 1); // for inserting zeroes to rep and def + size_t i = 0; // in the input (s.def/s.rep) + size_t empty_arrays = 0; + for (ssize_t row = 0; row < static_cast(offsets.size()); ++row) + { + size_t n = offsets[row] - offsets[row - 1]; + if (n) + { + /// Un-increment the first rep of the array. + /// Skip n "items" in the nested column; first element of each item has rep = 1 + /// (we incremented it above). + chassert(s.rep[i] == 1); + --s.rep[i]; + do + { + ++i; + if (i == s.rep.size()) + { + --n; + chassert(n == 0); + break; + } + n -= s.rep[i] == 1; + } while (n); + } + else + { + mask.push_back(1); + mask[i + empty_arrays] = 0; + ++empty_arrays; + } + } + + if (empty_arrays != 0) + { + expandDataByMask(s.def, mask, false); + expandDataByMask(s.rep, mask, false); + } +} + +parq::CompressionCodec::type compressionMethodToParquet(CompressionMethod c) +{ + switch (c) + { + case CompressionMethod::None: return parq::CompressionCodec::UNCOMPRESSED; + case CompressionMethod::Snappy: return parq::CompressionCodec::SNAPPY; + case CompressionMethod::Gzip: return parq::CompressionCodec::GZIP; + case CompressionMethod::Brotli: return parq::CompressionCodec::BROTLI; + case CompressionMethod::Lz4: return parq::CompressionCodec::LZ4_RAW; + case CompressionMethod::Zstd: return parq::CompressionCodec::ZSTD; + + default: + throw Exception(ErrorCodes::UNKNOWN_COMPRESSION_METHOD, "Compression method {} is not supported by Parquet", toContentEncodingName(c)); + } +} + +/// Depth-first traversal of the schema tree for this column. +void prepareColumnRecursive( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas); + +void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::string & name, + const WriteOptions & options, ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + /// Add physical column info. + auto & state = states.emplace_back(); + state.primitive_column = column; + state.compression = options.compression; + + state.column_chunk.__isset.meta_data = true; + state.column_chunk.meta_data.__set_path_in_schema({name}); + state.column_chunk.meta_data.__set_codec(compressionMethodToParquet(state.compression)); + + /// Add logical schema leaf. + auto & schema = schemas.emplace_back(); + schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + schema.__set_name(name); + + /// Convert the type enums. + + using T = parq::Type; + using C = parq::ConvertedType; + + auto types = [&](T::type type_, std::optional converted = std::nullopt, std::optional logical = std::nullopt) + { + state.column_chunk.meta_data.__set_type(type_); + schema.__set_type(type_); + if (converted) + schema.__set_converted_type(*converted); + if (logical) + schema.__set_logicalType(*logical); + }; + + auto int_type = [](Int8 bits, bool signed_) + { + parq::LogicalType t; + t.__isset.INTEGER = true; + t.INTEGER.__set_bitWidth(bits); + t.INTEGER.__set_isSigned(signed_); + return t; + }; + + auto fixed_string = [&](size_t size, std::optional converted = std::nullopt, std::optional logical = std::nullopt) + { + state.column_chunk.meta_data.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type_length(static_cast(size)); + if (converted) + schema.__set_converted_type(*converted); + if (logical) + schema.__set_logicalType(*logical); + }; + + auto decimal = [&](Int32 bytes, UInt32 precision, UInt32 scale) + { + state.column_chunk.meta_data.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type_length(bytes); + schema.__set_scale(static_cast(scale)); + schema.__set_precision(static_cast(precision)); + schema.__set_converted_type(parq::ConvertedType::DECIMAL); + parq::DecimalType d; + d.__set_scale(static_cast(scale)); + d.__set_precision(static_cast(precision)); + parq::LogicalType t; + t.__set_DECIMAL(d); + schema.__set_logicalType(t); + }; + + switch (type->getTypeId()) + { + case TypeIndex::UInt8: types(T::INT32, C::UINT_8 , int_type(8 , false)); break; + case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; + case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; + case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; + case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; + case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break; + case TypeIndex::Int32: types(T::INT32); break; + case TypeIndex::Int64: types(T::INT64); break; + case TypeIndex::Float32: types(T::FLOAT); break; + case TypeIndex::Float64: types(T::DOUBLE); break; + + /// These don't have suitable parquet logical types, so we write them as plain numbers. + /// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum + /// values in advance as part of the data type.) + case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8 + case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16 + case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 + case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16 + case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 + + case TypeIndex::Date32: + { + parq::LogicalType t; + t.__set_DATE({}); + types(T::INT32, C::DATE, t); + break; + } + + case TypeIndex::DateTime64: + { + std::optional converted; + std::optional unit; + switch (assert_cast(*type).getScale()) + { + case 3: + converted = parq::ConvertedType::TIMESTAMP_MILLIS; + unit.emplace().__set_MILLIS({}); + break; + case 6: + converted = parq::ConvertedType::TIMESTAMP_MICROS; + unit.emplace().__set_MICROS({}); + break; + case 9: + unit.emplace().__set_NANOS({}); + break; + } + + std::optional t; + if (unit) + { + parq::TimestampType tt; + tt.__set_isAdjustedToUTC(true); + tt.__set_unit(*unit); + t.emplace().__set_TIMESTAMP(tt); + } + types(T::INT64, converted, t); + break; + } + + case TypeIndex::String: + case TypeIndex::FixedString: + { + if (options.output_fixed_string_as_fixed_byte_array && + type->getTypeId() == TypeIndex::FixedString) + { + fixed_string(assert_cast(*type).getN()); + } + else if (options.output_string_as_string) + { + parq::LogicalType t; + t.__set_STRING({}); + types(T::BYTE_ARRAY, C::UTF8, t); + } + else + { + types(T::BYTE_ARRAY); + } + break; + } + + /// Parquet doesn't have logical types for these. + case TypeIndex::UInt128: fixed_string(16); break; + case TypeIndex::UInt256: fixed_string(32); break; + case TypeIndex::Int128: fixed_string(16); break; + case TypeIndex::Int256: fixed_string(32); break; + case TypeIndex::IPv6: fixed_string(16); break; + + case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break; + + default: + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of column '{}' is not supported for conversion into Parquet data format.", type->getFamilyName(), name); + } +} + +void prepareColumnNullable( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const ColumnNullable * column_nullable = assert_cast(column.get()); + ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); + DataTypePtr nested_type = assert_cast(type.get())->getNestedType(); + const NullMap & null_map = column_nullable->getNullMapData(); + + size_t child_states_begin = states.size(); + size_t child_schema_idx = schemas.size(); + + prepareColumnRecursive(nested_column, nested_type, name, options, states, schemas); + + if (schemas[child_schema_idx].repetition_type == parq::FieldRepetitionType::REQUIRED) + { + /// Normal case: we just slap a FieldRepetitionType::OPTIONAL onto the nested column. + schemas[child_schema_idx].repetition_type = parq::FieldRepetitionType::OPTIONAL; + } + else + { + /// Weird case: Nullable(Nullable(...)). Or Nullable(Tuple(Nullable(...))), etc. + /// This is probably not allowed in ClickHouse, but let's support it just in case. + auto & schema = *schemas.insert(schemas.begin() + child_schema_idx, {}); + schema.__set_repetition_type(parq::FieldRepetitionType::OPTIONAL); + schema.__set_name("nullable"); + schema.__set_num_children(1); + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), schema.name + "."); + } + } + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + auto & s = states[i]; + updateRepDefLevelsAndFilterColumnForNullable(s, null_map); + } +} + +void prepareColumnTuple( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_tuple = assert_cast(column.get()); + const auto * type_tuple = assert_cast(type.get()); + + auto & tuple_schema = schemas.emplace_back(); + tuple_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + tuple_schema.__set_name(name); + tuple_schema.__set_num_children(static_cast(type_tuple->getElements().size())); + + size_t child_states_begin = states.size(); + + for (size_t i = 0; i < type_tuple->getElements().size(); ++i) + prepareColumnRecursive(column_tuple->getColumnPtr(i), type_tuple->getElement(i), type_tuple->getNameByPosition(i + 1), options, states, schemas); + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + /// O(nesting_depth^2), but who cares. + path.insert(path.begin(), name); + } +} + +void prepareColumnArray( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_array = assert_cast(column.get()); + ColumnPtr nested_column = column_array->getDataPtr(); + DataTypePtr nested_type = assert_cast(type.get())->getNestedType(); + const auto & offsets = column_array->getOffsets(); + + /// Schema for lists https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + /// + /// required group `name` (List): + /// repeated group "list": + /// "element" + + /// Add the groups schema. + + schemas.emplace_back(); + schemas.emplace_back(); + auto & list_schema = schemas[schemas.size() - 2]; + auto & item_schema = schemas[schemas.size() - 1]; + + list_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + list_schema.__set_name(name); + list_schema.__set_num_children(1); + list_schema.__set_converted_type(parq::ConvertedType::LIST); + list_schema.__isset.logicalType = true; + list_schema.logicalType.__set_LIST({}); + + item_schema.__set_repetition_type(parq::FieldRepetitionType::REPEATED); + item_schema.__set_name("list"); + item_schema.__set_num_children(1); + + std::array path_prefix = {list_schema.name, item_schema.name}; + size_t child_states_begin = states.size(); + + /// Recurse. + prepareColumnRecursive(nested_column, nested_type, "element", options, states, schemas); + + /// Update repetition+definition levels and fully-qualified column names (x -> myarray.list.x). + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), path_prefix.begin(), path_prefix.end()); + + updateRepDefLevelsForArray(states[i], offsets); + } +} + +void prepareColumnMap( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_map = assert_cast(column.get()); + const auto * column_array = &column_map->getNestedColumn(); + const auto & offsets = column_array->getOffsets(); + ColumnPtr column_tuple = column_array->getDataPtr(); + + const auto * map_type = assert_cast(type.get()); + DataTypePtr tuple_type = std::make_shared(map_type->getKeyValueTypes(), Strings{"key", "value"}); + + /// Map is an array of tuples + /// https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps + /// + /// required group `name` (Map): + /// repeated group "key_value": + /// reqiured <...> "key" + /// <...> "value" + + auto & map_schema = schemas.emplace_back(); + map_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + map_schema.__set_name(name); + map_schema.__set_num_children(1); + map_schema.__set_converted_type(parq::ConvertedType::MAP); + map_schema.__set_logicalType({}); + map_schema.logicalType.__set_MAP({}); + + size_t tuple_schema_idx = schemas.size(); + size_t child_states_begin = states.size(); + + prepareColumnTuple(column_tuple, tuple_type, "key_value", options, states, schemas); + + schemas[tuple_schema_idx].__set_repetition_type(parq::FieldRepetitionType::REPEATED); + schemas[tuple_schema_idx].__set_converted_type(parq::ConvertedType::MAP_KEY_VALUE); + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), name); + + updateRepDefLevelsForArray(states[i], offsets); + } +} + +void prepareColumnRecursive( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + switch (type->getTypeId()) + { + case TypeIndex::Nullable: prepareColumnNullable(column, type, name, options, states, schemas); break; + case TypeIndex::Array: prepareColumnArray(column, type, name, options, states, schemas); break; + case TypeIndex::Tuple: prepareColumnTuple(column, type, name, options, states, schemas); break; + case TypeIndex::Map: prepareColumnMap(column, type, name, options, states, schemas); break; + case TypeIndex::LowCardinality: + { + auto nested_type = assert_cast(*type).getDictionaryType(); + if (nested_type->isNullable()) + prepareColumnNullable( + column->convertToFullColumnIfLowCardinality(), nested_type, name, options, states, schemas); + else + /// Use nested data type, but keep ColumnLowCardinality. The encoder can deal with it. + preparePrimitiveColumn(column, nested_type, name, options, states, schemas); + break; + } + default: + preparePrimitiveColumn(column, type, name, options, states, schemas); + break; + } +} + +} + +SchemaElements convertSchema(const Block & sample, const WriteOptions & options) +{ + SchemaElements schema; + auto & root = schema.emplace_back(); + root.__set_name("schema"); + root.__set_num_children(static_cast(sample.columns())); + + for (auto & c : sample) + prepareColumnForWrite(c.column, c.type, c.name, options, nullptr, &schema); + + return schema; +} + +void prepareColumnForWrite( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema) +{ + if (column->size() == 0 && out_columns_to_write != nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column passed to Parquet encoder"); + + ColumnChunkWriteStates states; + SchemaElements schemas; + prepareColumnRecursive(column, type, name, options, states, schemas); + + if (out_columns_to_write) + for (auto & s : states) + out_columns_to_write->push_back(std::move(s)); + if (out_schema) + out_schema->insert(out_schema->end(), schemas.begin(), schemas.end()); + + if (column->empty()) + states.clear(); +} + +} diff --git a/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp b/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp new file mode 100644 index 00000000000..2a99b028ae0 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB::Parquet +{ + +class WriteBufferTransport : public apache::thrift::transport::TTransport +{ +public: + WriteBuffer & out; + size_t bytes = 0; + + explicit WriteBufferTransport(WriteBuffer & out_) : out(out_) {} + + void write(const uint8_t* buf, uint32_t len) + { + out.write(reinterpret_cast(buf), len); + bytes += len; + } +}; + +template +size_t serializeThriftStruct(const T & obj, WriteBuffer & out) +{ + auto trans = std::make_shared(out); + auto proto = apache::thrift::protocol::TCompactProtocolFactoryT().getProtocol(trans); + obj.write(proto.get()); + return trans->bytes; +} + +template size_t serializeThriftStruct(const parquet::format::PageHeader &, WriteBuffer & out); +template size_t serializeThriftStruct(const parquet::format::ColumnChunk &, WriteBuffer & out); +template size_t serializeThriftStruct(const parquet::format::FileMetaData &, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/Parquet/ThriftUtil.h b/src/Processors/Formats/Impl/Parquet/ThriftUtil.h new file mode 100644 index 00000000000..1efbe0002d4 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ThriftUtil.h @@ -0,0 +1,17 @@ +#pragma once + +#include // in contrib/arrow/cpp/src/ , generated from parquet.thrift +#include + +namespace DB::Parquet +{ + +/// Returns number of bytes written. +template +size_t serializeThriftStruct(const T & obj, WriteBuffer & out); + +extern template size_t serializeThriftStruct(const parquet::format::PageHeader &, WriteBuffer & out); +extern template size_t serializeThriftStruct(const parquet::format::ColumnChunk &, WriteBuffer & out); +extern template size_t serializeThriftStruct(const parquet::format::FileMetaData &, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp new file mode 100644 index 00000000000..a29bb81f8dc --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -0,0 +1,816 @@ +#include "Processors/Formats/Impl/Parquet/Write.h" +#include "Processors/Formats/Impl/Parquet/ThriftUtil.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "config_version.h" + +namespace DB::ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int LIMIT_EXCEEDED; + extern const int LOGICAL_ERROR; +} + +namespace DB::Parquet +{ + +namespace parq = parquet::format; + +namespace +{ + +template +struct StatisticsNumeric +{ + T min = std::numeric_limits::max(); + T max = std::numeric_limits::min(); + + void add(SourceType x) + { + min = std::min(min, static_cast(x)); + max = std::max(max, static_cast(x)); + } + + void merge(const StatisticsNumeric & s) + { + min = std::min(min, s.min); + max = std::max(max, s.max); + } + + void clear() { *this = {}; } + + parq::Statistics get(const WriteOptions &) + { + parq::Statistics s; + s.__isset.min_value = s.__isset.max_value = true; + s.min_value.resize(sizeof(T)); + s.max_value.resize(sizeof(T)); + memcpy(s.min_value.data(), &min, sizeof(T)); + memcpy(s.max_value.data(), &max, sizeof(T)); + + if constexpr (std::is_signed::value) + { + s.__set_min(s.min_value); + s.__set_max(s.max_value); + } + return s; + } +}; + +struct StatisticsFixedString +{ + size_t fixed_string_size = UINT64_MAX; + const uint8_t * min = nullptr; + const uint8_t * max = nullptr; + + void add(parquet::FixedLenByteArray a) + { + chassert(fixed_string_size != UINT64_MAX); + addMin(a.ptr); + addMax(a.ptr); + } + + void merge(const StatisticsFixedString & s) + { + chassert(fixed_string_size == UINT64_MAX || fixed_string_size == s.fixed_string_size); + fixed_string_size = s.fixed_string_size; + if (s.min == nullptr) + return; + addMin(s.min); + addMax(s.max); + } + + void clear() { min = max = nullptr; } + + parq::Statistics get(const WriteOptions & options) + { + parq::Statistics s; + if (min == nullptr || fixed_string_size > options.max_statistics_size) + return s; + s.__set_min_value(std::string(reinterpret_cast(min), fixed_string_size)); + s.__set_max_value(std::string(reinterpret_cast(max), fixed_string_size)); + return s; + } + + void addMin(const uint8_t * p) + { + if (min == nullptr || memcmp(p, min, fixed_string_size) < 0) + min = p; + } + void addMax(const uint8_t * p) + { + if (max == nullptr || memcmp(p, max, fixed_string_size) > 0) + max = p; + } +}; + +struct StatisticsString +{ + parquet::ByteArray min; + parquet::ByteArray max; + + void add(parquet::ByteArray x) + { + addMin(x); + addMax(x); + } + + void merge(const StatisticsString & s) + { + if (s.min.ptr == nullptr) + return; + addMin(s.min); + addMax(s.max); + } + + void clear() { *this = {}; } + + parq::Statistics get(const WriteOptions & options) + { + parq::Statistics s; + if (min.ptr == nullptr) + return s; + if (static_cast(min.len) <= options.max_statistics_size) + s.__set_min_value(std::string(reinterpret_cast(min.ptr), static_cast(min.len))); + if (static_cast(max.len) <= options.max_statistics_size) + s.__set_max_value(std::string(reinterpret_cast(max.ptr), static_cast(max.len))); + return s; + } + + void addMin(parquet::ByteArray x) + { + if (min.ptr == nullptr || compare(x, min) < 0) + min = x; + } + + void addMax(parquet::ByteArray x) + { + if (max.ptr == nullptr || compare(x, max) > 0) + max = x; + } + + static int compare(parquet::ByteArray a, parquet::ByteArray b) + { + int t = memcmp(a.ptr, b.ptr, std::min(a.len, b.len)); + if (t != 0) + return t; + return a.len - b.len; + } +}; + +/// The column usually needs to be converted to one of Parquet physical types, e.g. UInt16 -> Int32 +/// or [element of ColumnString] -> std::string_view. +/// We do this conversion in small batches rather than all at once, just before encoding the batch, +/// in hopes of getting better performance through cache locality. +/// The Coverter* structs below are responsible for that. +/// When conversion is not needed, getBatch() will just return pointer into original data. + +template ::value, + To, + typename std::make_unsigned::type>::type> +struct ConverterNumeric +{ + using Statistics = StatisticsNumeric; + + const Col & column; + PODArray buf; + + explicit ConverterNumeric(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const To * getBatch(size_t offset, size_t count) + { + if constexpr (sizeof(*column.getData().data()) == sizeof(To)) + return reinterpret_cast(column.getData().data() + offset); + else + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i] = static_cast(column.getData()[offset + i]); + return buf.data(); + } + } +}; + +struct ConverterString +{ + using Statistics = StatisticsString; + + const ColumnString & column; + PODArray buf; + + explicit ConverterString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::ByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + { + StringRef s = column.getDataAt(offset + i); + buf[i] = parquet::ByteArray(static_cast(s.size), reinterpret_cast(s.data)); + } + return buf.data(); + } +}; + +struct ConverterFixedString +{ + using Statistics = StatisticsFixedString; + + const ColumnFixedString & column; + PODArray buf; + + explicit ConverterFixedString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i].ptr = reinterpret_cast(column.getChars().data() + (offset + i) * column.getN()); + return buf.data(); + } + + size_t fixedStringSize() { return column.getN(); } +}; + +struct ConverterFixedStringAsString +{ + using Statistics = StatisticsString; + + const ColumnFixedString & column; + PODArray buf; + + explicit ConverterFixedStringAsString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::ByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i] = parquet::ByteArray(static_cast(column.getN()), reinterpret_cast(column.getChars().data() + (offset + i) * column.getN())); + return buf.data(); + } +}; + +template +struct ConverterNumberAsFixedString +{ + /// Calculate min/max statistics for little-endian fixed strings, not numbers, because parquet + /// doesn't know it's numbers. + using Statistics = StatisticsFixedString; + + const ColumnVector & column; + PODArray buf; + + explicit ConverterNumberAsFixedString(const ColumnPtr & c) : column(assert_cast &>(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i].ptr = reinterpret_cast(column.getData().data() + offset + i); + return buf.data(); + } + + size_t fixedStringSize() { return sizeof(T); } +}; + +/// Like ConverterNumberAsFixedString, but converts to big-endian. Because that's the byte order +/// Parquet uses for decimal types and literally nothing else, for some reason. +template +struct ConverterDecimal +{ + using Statistics = StatisticsFixedString; + + const ColumnDecimal & column; + PODArray data_buf; + PODArray ptr_buf; + + explicit ConverterDecimal(const ColumnPtr & c) : column(assert_cast &>(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + data_buf.resize(count * sizeof(T)); + ptr_buf.resize(count); + memcpy(data_buf.data(), reinterpret_cast(column.getData().data() + offset), count * sizeof(T)); + for (size_t i = 0; i < count; ++i) + { + std::reverse(data_buf.data() + i * sizeof(T), data_buf.data() + (i + 1) * sizeof(T)); + ptr_buf[i].ptr = data_buf.data() + i * sizeof(T); + } + return ptr_buf.data(); + } + + size_t fixedStringSize() { return sizeof(T); } +}; + +/// Returns either `source` or `scratch`. +PODArray & compress(PODArray & source, PODArray & scratch, CompressionMethod method) +{ + /// We could use wrapWriteBufferWithCompressionMethod() for everything, but I worry about the + /// overhead of creating a bunch of WriteBuffers on each page (thousands of values). + switch (method) + { + case CompressionMethod::None: + return source; + + case CompressionMethod::Lz4: + { + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wold-style-cast" + + size_t max_dest_size = LZ4_COMPRESSBOUND(source.size()); + + #pragma clang diagnostic pop + + if (max_dest_size > std::numeric_limits::max()) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", formatReadableSizeWithBinarySuffix(source.size())); + + scratch.resize(max_dest_size); + + int compressed_size = LZ4_compress_default( + source.data(), + scratch.data(), + static_cast(source.size()), + static_cast(max_dest_size)); + + scratch.resize(static_cast(compressed_size)); + return scratch; + } + + default: + { + auto dest_buf = std::make_unique>>(scratch); + auto compressed_buf = wrapWriteBufferWithCompressionMethod( + std::move(dest_buf), + method, + /*level*/ 3, + source.size(), + /*existing_memory*/ source.data()); + chassert(compressed_buf->position() == source.data()); + chassert(compressed_buf->available() == source.size()); + compressed_buf->position() += source.size(); + compressed_buf->finalize(); + return scratch; + } + } +} + +void encodeRepDefLevelsRLE(const UInt8 * data, size_t size, UInt8 max_level, PODArray & out) +{ + using arrow::util::RleEncoder; + + chassert(max_level > 0); + size_t offset = out.size(); + size_t prefix_size = sizeof(Int32); + + int bit_width = bitScanReverse(max_level) + 1; + int max_rle_size = RleEncoder::MaxBufferSize(bit_width, static_cast(size)) + + RleEncoder::MinBufferSize(bit_width); + + out.resize(offset + prefix_size + max_rle_size); + + RleEncoder encoder(reinterpret_cast(out.data() + offset + prefix_size), max_rle_size, bit_width); + for (size_t i = 0; i < size; ++i) + encoder.Put(data[i]); + encoder.Flush(); + Int32 len = encoder.len(); + + memcpy(out.data() + offset, &len, prefix_size); + out.resize(offset + prefix_size + len); +} + +void addToEncodingsUsed(ColumnChunkWriteState & s, parq::Encoding::type e) +{ + if (!std::count(s.column_chunk.meta_data.encodings.begin(), s.column_chunk.meta_data.encodings.end(), e)) + s.column_chunk.meta_data.encodings.push_back(e); +} + +void writePage(const parq::PageHeader & header, const PODArray & compressed, ColumnChunkWriteState & s, WriteBuffer & out) +{ + size_t header_size = serializeThriftStruct(header, out); + out.write(compressed.data(), compressed.size()); + + /// Remember first data page and first dictionary page. + if (header.__isset.data_page_header && s.column_chunk.meta_data.data_page_offset == -1) + s.column_chunk.meta_data.__set_data_page_offset(s.column_chunk.meta_data.total_compressed_size); + if (header.__isset.dictionary_page_header && !s.column_chunk.meta_data.__isset.dictionary_page_offset) + s.column_chunk.meta_data.__set_dictionary_page_offset(s.column_chunk.meta_data.total_compressed_size); + + s.column_chunk.meta_data.total_uncompressed_size += header.uncompressed_page_size + header_size; + s.column_chunk.meta_data.total_compressed_size += header.compressed_page_size + header_size; +} + +template +void writeColumnImpl( + ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out, Converter && converter) +{ + size_t num_values = s.max_def > 0 ? s.def.size() : s.primitive_column->size(); + auto encoding = options.encoding; + + typename Converter::Statistics page_statistics; + typename Converter::Statistics total_statistics; + + /// We start with dictionary encoding, then switch to `encoding` (non-dictionary) if the + /// dictionary gets too big. That's how arrow does it too. + bool initially_used_dictionary = options.use_dictionary_encoding; + bool currently_using_dictionary = initially_used_dictionary; + + std::optional fixed_string_descr; + if constexpr (std::is_same::value) + { + /// This just communicates one number to MakeTypedEncoder(): the fixed string length. + fixed_string_descr.emplace(parquet::schema::PrimitiveNode::Make( + "", parquet::Repetition::REQUIRED, parquet::Type::FIXED_LEN_BYTE_ARRAY, + parquet::ConvertedType::NONE, static_cast(converter.fixedStringSize())), 0, 0); + + page_statistics.fixed_string_size = converter.fixedStringSize(); + } + + /// Could use an arena here (by passing a custom MemoryPool), to reuse memory across pages. + /// Alternatively, we could avoid using arrow's dictionary encoding code and leverage + /// ColumnLowCardinality instead. It would work basically the same way as what this function + /// currently does: add values to the ColumnRowCardinality (instead of `encoder`) in batches, + /// checking dictionary size after each batch; if it gets big, flush the dictionary and the + /// indices and switch to non-dictionary encoding. Feels like it could even be slightly less code. + auto encoder = parquet::MakeTypedEncoder( + // ignored if using dictionary + static_cast(encoding), + currently_using_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); + + struct PageData + { + parq::PageHeader header; + PODArray data; + }; + std::vector dict_encoded_pages; // can't write them out until we have full dictionary + + /// Reused across pages to reduce number of allocations and improve locality. + PODArray encoded; + PODArray compressed_maybe; + + /// Start of current page. + size_t def_offset = 0; // index in def and rep + size_t data_offset = 0; // index in primitive_column + + auto flush_page = [&](size_t def_count, size_t data_count) + { + encoded.clear(); + + /// Concatenate encoded rep, def, and data. + + if (s.max_rep > 0) + encodeRepDefLevelsRLE(s.rep.data() + def_offset, def_count, s.max_rep, encoded); + if (s.max_def > 0) + encodeRepDefLevelsRLE(s.def.data() + def_offset, def_count, s.max_def, encoded); + + std::shared_ptr values = encoder->FlushValues(); // resets it for next page + + encoded.resize(encoded.size() + values->size()); + memcpy(encoded.data() + encoded.size() - values->size(), values->data(), values->size()); + values.reset(); + + if (encoded.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Uncompressed page is too big: {}", encoded.size()); + + size_t uncompressed_size = encoded.size(); + auto & compressed = compress(encoded, compressed_maybe, s.compression); + + if (compressed.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Compressed page is too big: {}", compressed.size()); + + parq::PageHeader header; + header.__set_type(parq::PageType::DATA_PAGE); + header.__set_uncompressed_page_size(static_cast(uncompressed_size)); + header.__set_compressed_page_size(static_cast(compressed.size())); + header.__isset.data_page_header = true; + auto & d = header.data_page_header; + d.__set_num_values(static_cast(def_count)); + d.__set_encoding(currently_using_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); + d.__set_definition_level_encoding(parq::Encoding::RLE); + d.__set_repetition_level_encoding(parq::Encoding::RLE); + /// We could also put checksum in `header.crc`, but apparently no one uses it: + /// https://issues.apache.org/jira/browse/PARQUET-594 + + if (options.write_page_statistics) + { + d.__set_statistics(page_statistics.get(options)); + + if (s.max_def == 1 && s.max_rep == 0) + d.statistics.__set_null_count(static_cast(def_count - data_count)); + } + + total_statistics.merge(page_statistics); + page_statistics.clear(); + + if (currently_using_dictionary) + { + dict_encoded_pages.push_back({.header = std::move(header)}); + std::swap(dict_encoded_pages.back().data, compressed); + } + else + { + writePage(header, compressed, s, out); + } + + def_offset += def_count; + data_offset += data_count; + }; + + auto flush_dict = [&] -> bool + { + auto * dict_encoder = dynamic_cast *>(encoder.get()); + int dict_size = dict_encoder->dict_encoded_size(); + + encoded.resize(static_cast(dict_size)); + dict_encoder->WriteDict(reinterpret_cast(encoded.data())); + + auto & compressed = compress(encoded, compressed_maybe, s.compression); + + if (compressed.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Compressed dictionary page is too big: {}", compressed.size()); + + parq::PageHeader header; + header.__set_type(parq::PageType::DICTIONARY_PAGE); + header.__set_uncompressed_page_size(dict_size); + header.__set_compressed_page_size(static_cast(compressed.size())); + header.__isset.dictionary_page_header = true; + header.dictionary_page_header.__set_num_values(dict_encoder->num_entries()); + header.dictionary_page_header.__set_encoding(parq::Encoding::PLAIN); + + writePage(header, compressed, s, out); + + for (auto & p : dict_encoded_pages) + writePage(p.header, p.data, s, out); + + dict_encoded_pages.clear(); + encoder.reset(); + + return true; + }; + + auto is_dict_too_big = [&] { + auto * dict_encoder = dynamic_cast *>(encoder.get()); + int dict_size = dict_encoder->dict_encoded_size(); + return static_cast(dict_size) >= options.dictionary_size_limit; + }; + + while (def_offset < num_values) + { + /// Pick enough data for a page. + size_t next_def_offset = def_offset; + size_t next_data_offset = data_offset; + while (true) + { + /// Bite off a batch of defs and corresponding data values. + size_t def_count = std::min(options.write_batch_size, num_values - next_def_offset); + size_t data_count = 0; + if (s.max_def == 0) + data_count = def_count; + else + for (size_t i = 0; i < def_count; ++i) + data_count += s.def[next_def_offset + i] == s.max_def; + + /// Encode the data (but not the levels yet), so that we can estimate its encoded size. + const typename ParquetDType::c_type * converted = converter.getBatch(next_data_offset, data_count); + + if (options.write_page_statistics || options.write_column_chunk_statistics) + for (size_t i = 0; i < data_count; ++i) + page_statistics.add(converted[i]); + + encoder->Put(converted, static_cast(data_count)); + + next_def_offset += def_count; + next_data_offset += data_count; + + if (currently_using_dictionary && is_dict_too_big()) + { + /// Fallback to non-dictionary encoding. + flush_page(next_def_offset - def_offset, next_data_offset - data_offset); + flush_dict(); + + currently_using_dictionary = false; + encoder = parquet::MakeTypedEncoder( + static_cast(encoding)); + break; + } + + if (next_def_offset == num_values || + static_cast(encoder->EstimatedDataEncodedSize()) >= options.data_page_size) + { + flush_page(next_def_offset - def_offset, next_data_offset - data_offset); + break; + } + } + } + + if (currently_using_dictionary) + flush_dict(); + + chassert(data_offset == s.primitive_column->size()); + + if (options.write_column_chunk_statistics) + { + s.column_chunk.meta_data.__set_statistics(total_statistics.get(options)); + + if (s.max_def == 1 && s.max_rep == 0) + s.column_chunk.meta_data.statistics.__set_null_count(static_cast(def_offset - data_offset)); + } + + /// Report which encodings we've used. + if (s.max_rep > 0 || s.max_def > 0) + addToEncodingsUsed(s, parq::Encoding::RLE); // levels + if (!currently_using_dictionary) + addToEncodingsUsed(s, encoding); // non-dictionary encoding + if (initially_used_dictionary) + { + addToEncodingsUsed(s, parq::Encoding::PLAIN); // dictionary itself + addToEncodingsUsed(s, parq::Encoding::RLE_DICTIONARY); // ids + } +} + +} + +void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out) +{ + s.column_chunk.meta_data.__set_num_values(s.max_def > 0 ? s.def.size() : s.primitive_column->size()); + + /// We'll be updating these as we go. + s.column_chunk.meta_data.__set_encodings({}); + s.column_chunk.meta_data.__set_total_compressed_size(0); + s.column_chunk.meta_data.__set_total_uncompressed_size(0); + s.column_chunk.meta_data.__set_data_page_offset(-1); + + s.primitive_column = s.primitive_column->convertToFullColumnIfLowCardinality(); + + switch (s.primitive_column->getDataType()) + { + /// Numeric conversion to Int32 or Int64. + #define N(source_type, parquet_dtype) \ + writeColumnImpl(s, options, out, \ + ConverterNumeric, parquet::parquet_dtype::c_type>( \ + s.primitive_column)) + + case TypeIndex::UInt8 : N(UInt8 , Int32Type); break; + case TypeIndex::UInt16 : N(UInt16, Int32Type); break; + case TypeIndex::UInt32 : N(UInt32, Int32Type); break; + case TypeIndex::UInt64 : N(UInt64, Int64Type); break; + case TypeIndex::Int8 : N(Int8 , Int32Type); break; + case TypeIndex::Int16 : N(Int16 , Int32Type); break; + case TypeIndex::Int32 : N(Int32 , Int32Type); break; + case TypeIndex::Int64 : N(Int64 , Int64Type); break; + + case TypeIndex::Enum8: N(Int8 , Int32Type); break; + case TypeIndex::Enum16: N(Int16 , Int32Type); break; + case TypeIndex::Date: N(UInt16, Int32Type); break; + case TypeIndex::Date32: N(Int32 , Int32Type); break; + case TypeIndex::DateTime: N(UInt32, Int32Type); break; + + #undef N + + case TypeIndex::Float32: + writeColumnImpl( + s, options, out, ConverterNumeric, Float32, Float32>( + s.primitive_column)); + break; + + case TypeIndex::Float64: + writeColumnImpl( + s, options, out, ConverterNumeric, Float64, Float64>( + s.primitive_column)); + break; + + case TypeIndex::DateTime64: + writeColumnImpl( + s, options, out, ConverterNumeric, Int64, Int64>( + s.primitive_column)); + break; + + case TypeIndex::IPv4: + writeColumnImpl( + s, options, out, ConverterNumeric, Int32, UInt32>( + s.primitive_column)); + break; + + case TypeIndex::String: + writeColumnImpl( + s, options, out, ConverterString(s.primitive_column)); + break; + + case TypeIndex::FixedString: + if (options.output_fixed_string_as_fixed_byte_array) + writeColumnImpl( + s, options, out, ConverterFixedString(s.primitive_column)); + else + writeColumnImpl( + s, options, out, ConverterFixedStringAsString(s.primitive_column)); + break; + + #define F(source_type) \ + writeColumnImpl( \ + s, options, out, ConverterNumberAsFixedString(s.primitive_column)) + case TypeIndex::UInt128: F(UInt128); break; + case TypeIndex::UInt256: F(UInt256); break; + case TypeIndex::Int128: F(Int128); break; + case TypeIndex::Int256: F(Int256); break; + case TypeIndex::IPv6: F(IPv6); break; + #undef F + + #define D(source_type) \ + writeColumnImpl( \ + s, options, out, ConverterDecimal(s.primitive_column)) + case TypeIndex::Decimal32: D(Decimal32); break; + case TypeIndex::Decimal64: D(Decimal64); break; + case TypeIndex::Decimal128: D(Decimal128); break; + case TypeIndex::Decimal256: D(Decimal256); break; + #undef D + + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column type: {}", s.primitive_column->getFamilyName()); + } + + /// Free some memory. + s.primitive_column = {}; + s.def = {}; + s.rep = {}; +} + +void writeFileHeader(WriteBuffer & out) +{ + /// Write the magic bytes. We're a wizard now. + out.write("PAR1", 4); +} + +parq::ColumnChunk finalizeColumnChunkAndWriteFooter( + size_t offset_in_file, ColumnChunkWriteState s, const WriteOptions &, WriteBuffer & out) +{ + if (s.column_chunk.meta_data.data_page_offset != -1) + s.column_chunk.meta_data.data_page_offset += offset_in_file; + if (s.column_chunk.meta_data.__isset.dictionary_page_offset) + s.column_chunk.meta_data.dictionary_page_offset += offset_in_file; + s.column_chunk.file_offset = offset_in_file + s.column_chunk.meta_data.total_compressed_size; + + serializeThriftStruct(s.column_chunk, out); + + return std::move(s.column_chunk); +} + +parq::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows) +{ + parq::RowGroup r; + r.__set_num_rows(num_rows); + r.__set_columns(std::move(column_chunks)); + r.__set_total_compressed_size(0); + for (auto & c : r.columns) + { + r.total_byte_size += c.meta_data.total_uncompressed_size; + r.total_compressed_size += c.meta_data.total_compressed_size; + } + if (!r.columns.empty()) + { + auto & m = r.columns[0].meta_data; + r.__set_file_offset(m.__isset.dictionary_page_offset ? m.dictionary_page_offset : m.data_page_offset); + } + return r; +} + +void writeFileFooter(std::vector row_groups, SchemaElements schema, const WriteOptions & options, WriteBuffer & out) +{ + parq::FileMetaData meta; + meta.version = 2; + meta.schema = std::move(schema); + meta.row_groups = std::move(row_groups); + for (auto & r : meta.row_groups) + meta.num_rows += r.num_rows; + meta.__set_created_by(VERSION_NAME " " VERSION_DESCRIBE); + + if (options.write_page_statistics || options.write_column_chunk_statistics) + { + meta.__set_column_orders({}); + for (auto & s : meta.schema) + if (!s.__isset.num_children) + meta.column_orders.emplace_back(); + for (auto & c : meta.column_orders) + c.__set_TYPE_ORDER({}); + } + + size_t footer_size = serializeThriftStruct(meta, out); + + if (footer_size > INT32_MAX) + throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Parquet file metadata too big: {}", footer_size); + + writeIntBinary(static_cast(footer_size), out); + out.write("PAR1", 4); +} + +} diff --git a/src/Processors/Formats/Impl/Parquet/Write.h b/src/Processors/Formats/Impl/Parquet/Write.h new file mode 100644 index 00000000000..333a32e191f --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/Write.h @@ -0,0 +1,135 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB::Parquet +{ + +/// A good resource for learning how Parquet format works is +/// contrib/arrow/cpp/src/parquet/parquet.thrift + +struct WriteOptions +{ + bool output_string_as_string = false; + bool output_fixed_string_as_fixed_byte_array = true; + + CompressionMethod compression = CompressionMethod::Lz4; + + size_t data_page_size = 1024 * 1024; + size_t write_batch_size = 1024; + + bool use_dictionary_encoding = true; + size_t dictionary_size_limit = 1024 * 1024; + /// If using dictionary, this encoding is used as a fallback when dictionary gets too big. + /// Otherwise, this is used for everything. + parquet::format::Encoding::type encoding = parquet::format::Encoding::PLAIN; + + bool write_page_statistics = true; + bool write_column_chunk_statistics = true; + size_t max_statistics_size = 4096; +}; + +/// Information about a primitive column (leaf of the schema tree) to write to Parquet file. +struct ColumnChunkWriteState +{ + /// After writeColumnChunkBody(), offsets in this struct are relative to the start of column chunk. + /// Then finalizeColumnChunkAndWriteFooter() fixes them up before writing to file. + parquet::format::ColumnChunk column_chunk; + + ColumnPtr primitive_column; + CompressionMethod compression; // must match what's inside column_chunk + + /// Repetition and definition levels. Produced by prepareColumnForWrite(). + /// def is empty iff max_def == 0, which means no arrays or nullables. + /// rep is empty iff max_rep == 0, which means no arrays. + PaddedPODArray def; // definition levels + PaddedPODArray rep; // repetition levels + /// Max possible levels, according to schema. Actual max in def/rep may be smaller. + UInt8 max_def = 0; + UInt8 max_rep = 0; + + ColumnChunkWriteState() = default; + /// Prevent accidental copying. + ColumnChunkWriteState(ColumnChunkWriteState &&) = default; + ColumnChunkWriteState & operator=(ColumnChunkWriteState &&) = default; + + /// Estimated memory usage. + size_t allocatedBytes() const + { + size_t r = def.allocated_bytes() + rep.allocated_bytes(); + if (primitive_column) + r += primitive_column->allocatedBytes(); + return r; + } +}; + +using SchemaElements = std::vector; +using ColumnChunkWriteStates = std::vector; + +/// Parquet file consists of row groups, which consist of column chunks. +/// +/// Column chunks can be encoded mostly independently of each other, in parallel. +/// But there are two small complications: +/// 1. One ClickHouse column can translate to multiple leaf columns in parquet. +/// E.g. tuples and maps. +/// If all primitive columns are in one big tuple, we'd like to encode them in parallel too, +/// even though they're one top-level ClickHouse column. +/// 2. At the end of each encoded column chunk there's a footer (struct ColumnMetaData) that +/// contains some absolute offsets in the file. We can't encode it until we know the exact +/// position in the file where the column chunk will go. So these footers have to be serialized +/// sequentially, after we know sizes of all previous column chunks. +/// +/// With that in mind, here's how to write a parquet file: +/// +/// (1) writeFileHeader() +/// (2) For each row group: +/// | (3) For each ClickHouse column: +/// | (4) Call prepareColumnForWrite(). +/// | It'll produce one or more ColumnChunkWriteStates, corresponding to primitive columns that +/// | we need to write. +/// | It'll also produce SchemaElements as a byproduct, describing the logical types and +/// | groupings of the physical columns (e.g. tuples, arrays, maps). +/// | (5) For each ColumnChunkWriteState: +/// | (6) Call writeColumnChunkBody() to write the actual data to the given WriteBuffer. +/// | (7) Call finalizeColumnChunkAndWriteFooter() to write the footer of the column chunk. +/// | (8) Call makeRowGroup() using the ColumnChunk metadata structs from previous step. +/// (9) Call writeFileFooter() using the row groups from previous step and SchemaElements from +/// convertSchema(). +/// +/// Steps (4) and (6) can be parallelized, both within and across row groups. + +/// Parquet schema is a tree of SchemaElements, flattened into a list in depth-first order. +/// Leaf nodes correspond to physical columns of primitive types. Inner nodes describe logical +/// groupings of those columns, e.g. tuples or structs. +SchemaElements convertSchema(const Block & sample, const WriteOptions & options); + +void prepareColumnForWrite( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema = nullptr); + +void writeFileHeader(WriteBuffer & out); + +/// Encodes a column chunk, without the footer. +/// The ColumnChunkWriteState-s should then passed to finalizeColumnChunkAndWriteFooter(). +void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out); + +/// Unlike most of the column chunk data, the footer (`ColumnMetaData`) needs to know its absolute +/// offset in the file. So we encode it separately, after all previous row groups and column chunks +/// have been encoded. +/// (If you're wondering if the 8-byte offset values can be patched inside the encoded blob - no, +/// they're varint-encoded and can't be padded to a fixed length.) +/// `offset_in_file` is the absolute position in the file where the writeColumnChunkBody()'s output +/// starts. +/// Returns a ColumnChunk to add to the RowGroup. +parquet::format::ColumnChunk finalizeColumnChunkAndWriteFooter( + size_t offset_in_file, ColumnChunkWriteState s, const WriteOptions & options, WriteBuffer & out); + +parquet::format::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows); + +void writeFileFooter(std::vector row_groups, SchemaElements schema, const WriteOptions & options, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 91840cd2c50..9a2d9072860 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -3,14 +3,23 @@ #if USE_PARQUET #include +#include #include #include "ArrowBufferedStreams.h" #include "CHColumnToArrowColumn.h" +namespace CurrentMetrics +{ + extern const Metric ParquetEncoderThreads; + extern const Metric ParquetEncoderThreadsActive; +} + namespace DB { +using namespace Parquet; + namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; @@ -67,11 +76,219 @@ namespace ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_), format_settings{format_settings_} { + if (format_settings.parquet.use_custom_encoder) + { + if (format_settings.parquet.parallel_encoding && format_settings.max_threads > 1) + pool = std::make_unique( + CurrentMetrics::ParquetEncoderThreads, CurrentMetrics::ParquetEncoderThreadsActive, + format_settings.max_threads); + + using C = FormatSettings::ParquetCompression; + switch (format_settings.parquet.output_compression_method) + { + case C::NONE: options.compression = CompressionMethod::None; break; + case C::SNAPPY: options.compression = CompressionMethod::Snappy; break; + case C::ZSTD: options.compression = CompressionMethod::Zstd; break; + case C::LZ4: options.compression = CompressionMethod::Lz4; break; + case C::GZIP: options.compression = CompressionMethod::Gzip; break; + case C::BROTLI: options.compression = CompressionMethod::Brotli; break; + } + options.output_string_as_string = format_settings.parquet.output_string_as_string; + options.output_fixed_string_as_fixed_byte_array = format_settings.parquet.output_fixed_string_as_fixed_byte_array; + options.data_page_size = format_settings.parquet.data_page_size; + options.write_batch_size = format_settings.parquet.write_batch_size; + + schema = convertSchema(header_, options); + } } -void ParquetBlockOutputFormat::consumeStaged() +ParquetBlockOutputFormat::~ParquetBlockOutputFormat() { - const size_t columns_num = staging_chunks.at(0).getNumColumns(); + if (pool) + { + is_stopped = true; + pool->wait(); + } +} + +void ParquetBlockOutputFormat::consume(Chunk chunk) +{ + /// Poll background tasks. + if (pool) + { + std::unique_lock lock(mutex); + while (true) + { + /// If some row groups are ready to be written to the file, write them. + reapCompletedRowGroups(lock); + + if (background_exception) + std::rethrow_exception(background_exception); + + if (is_stopped) + return; + + /// If there's too much work in flight, wait for some of it to complete. + if (row_groups.size() < 2) + break; + if (bytes_in_flight <= format_settings.parquet.row_group_bytes * 4 && + task_queue.size() <= format_settings.max_threads * 4) + break; + + condvar.wait(lock); + } + } + + /// Do something like SquashingTransform to produce big enough row groups. + /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. + /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more + /// convenient to do the squashing here. It's also parallelized here. + + if (chunk.getNumRows() != 0) + { + staging_rows += chunk.getNumRows(); + staging_bytes += chunk.bytes(); + staging_chunks.push_back(std::move(chunk)); + } + + const size_t target_rows = std::max(static_cast(1), format_settings.parquet.row_group_rows); + + if (staging_rows < target_rows && + staging_bytes < format_settings.parquet.row_group_bytes) + return; + + /// In the rare case that more than `row_group_rows` rows arrived in one chunk, split the + /// staging chunk into multiple row groups. + if (staging_rows >= target_rows * 2) + { + /// Increase row group size slightly (by < 2x) to avoid a small row group at the end. + size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); + size_t row_group_size = (staging_rows - 1) / num_row_groups + 1; // round up + + Chunk concatenated = std::move(staging_chunks[0]); + for (size_t i = 1; i < staging_chunks.size(); ++i) + concatenated.append(staging_chunks[i]); + staging_chunks.clear(); + + for (size_t offset = 0; offset < staging_rows; offset += row_group_size) + { + size_t count = std::min(row_group_size, staging_rows - offset); + MutableColumns columns = concatenated.cloneEmptyColumns(); + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count); + + Chunks piece; + piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo()); + writeRowGroup(std::move(piece)); + } + } + else + { + writeRowGroup(std::move(staging_chunks)); + } + + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; +} + +void ParquetBlockOutputFormat::finalizeImpl() +{ + if (!staging_chunks.empty()) + writeRowGroup(std::move(staging_chunks)); + + if (format_settings.parquet.use_custom_encoder) + { + if (pool) + { + std::unique_lock lock(mutex); + + /// Wait for background work to complete. + while (true) + { + reapCompletedRowGroups(lock); + + if (background_exception) + std::rethrow_exception(background_exception); + + if (is_stopped) + return; + + if (row_groups.empty()) + break; + + condvar.wait(lock); + } + } + + if (row_groups_complete.empty()) + writeFileHeader(out); + writeFileFooter(std::move(row_groups_complete), schema, options, out); + } + else + { + if (!file_writer) + { + Block header = materializeBlock(getPort(PortKind::Main).getHeader()); + std::vector chunks; + chunks.push_back(Chunk(header.getColumns(), 0)); + writeRowGroup(std::move(chunks)); + } + + if (file_writer) + { + auto status = file_writer->Close(); + if (!status.ok()) + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); + } + } +} + +void ParquetBlockOutputFormat::resetFormatterImpl() +{ + if (pool) + { + is_stopped = true; + pool->wait(); + is_stopped = false; + } + + background_exception = nullptr; + threads_running = 0; + task_queue.clear(); + row_groups.clear(); + file_writer.reset(); + row_groups_complete.clear(); + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; +} + +void ParquetBlockOutputFormat::onCancel() +{ + is_stopped = true; +} + +void ParquetBlockOutputFormat::writeRowGroup(std::vector chunks) +{ + if (pool) + writeRowGroupInParallel(std::move(chunks)); + else if (!format_settings.parquet.use_custom_encoder) + writeUsingArrow(std::move(chunks)); + else + { + Chunk concatenated = std::move(chunks[0]); + for (size_t i = 1; i < chunks.size(); ++i) + concatenated.append(chunks[i]); + chunks.clear(); + + writeRowGroupInOneThread(std::move(concatenated)); + } +} + +void ParquetBlockOutputFormat::writeUsingArrow(std::vector chunks) +{ + const size_t columns_num = chunks.at(0).getNumColumns(); std::shared_ptr arrow_table; if (!ch_column_to_arrow_column) @@ -85,7 +302,7 @@ void ParquetBlockOutputFormat::consumeStaged() format_settings.parquet.output_fixed_string_as_fixed_byte_array); } - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, staging_chunks, columns_num); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunks, columns_num); if (!file_writer) { @@ -112,64 +329,228 @@ void ParquetBlockOutputFormat::consumeStaged() file_writer = std::move(result.ValueOrDie()); } - // TODO: calculate row_group_size depending on a number of rows and table size - - // allow slightly bigger than row_group_size to avoid a very small tail row group - auto status = file_writer->WriteTable(*arrow_table, std::max(format_settings.parquet.row_group_rows, staging_rows)); + auto status = file_writer->WriteTable(*arrow_table, INT64_MAX); if (!status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while writing a table: {}", status.ToString()); } -void ParquetBlockOutputFormat::consume(Chunk chunk) +void ParquetBlockOutputFormat::writeRowGroupInOneThread(Chunk chunk) { - /// Do something like SquashingTransform to produce big enough row groups. - /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. - /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more - /// convenient to do the squashing here. - staging_rows += chunk.getNumRows(); - staging_bytes += chunk.bytes(); - staging_chunks.push_back(std::move(chunk)); - chassert(staging_chunks.back().getNumColumns() == staging_chunks.front().getNumColumns()); - if (staging_rows < format_settings.parquet.row_group_rows && - staging_bytes < format_settings.parquet.row_group_bytes) - { + if (chunk.getNumRows() == 0) return; - } - else + + const Block & header = getPort(PortKind::Main).getHeader(); + Parquet::ColumnChunkWriteStates columns_to_write; + chassert(header.columns() == chunk.getNumColumns()); + for (size_t i = 0; i < header.columns(); ++i) + prepareColumnForWrite( + chunk.getColumns()[i], header.getByPosition(i).type, header.getByPosition(i).name, + options, &columns_to_write); + + if (row_groups_complete.empty()) + writeFileHeader(out); + + std::vector column_chunks; + for (auto & s : columns_to_write) { - consumeStaged(); - staging_chunks.clear(); - staging_rows = 0; - staging_bytes = 0; + size_t offset = out.count(); + writeColumnChunkBody(s, options, out); + auto c = finalizeColumnChunkAndWriteFooter(offset, std::move(s), options, out); + column_chunks.push_back(std::move(c)); + } + + auto r = makeRowGroup(std::move(column_chunks), chunk.getNumRows()); + row_groups_complete.push_back(std::move(r)); +} + +void ParquetBlockOutputFormat::writeRowGroupInParallel(std::vector chunks) +{ + std::unique_lock lock(mutex); + + const Block & header = getPort(PortKind::Main).getHeader(); + + RowGroupState & r = row_groups.emplace_back(); + r.column_chunks.resize(header.columns()); + r.tasks_in_flight = r.column_chunks.size(); + + std::vector columnses; + for (auto & chunk : chunks) + { + chassert(header.columns() == chunk.getNumColumns()); + r.num_rows += chunk.getNumRows(); + columnses.push_back(chunk.detachColumns()); + } + + for (size_t i = 0; i < header.columns(); ++i) + { + Task & t = task_queue.emplace_back(&r, i, this); + t.column_type = header.getByPosition(i).type; + t.column_name = header.getByPosition(i).name; + + /// Defer concatenating the columns to the threads. + size_t bytes = 0; + for (size_t j = 0; j < chunks.size(); ++j) + { + auto & col = columnses[j][i]; + bytes += col->allocatedBytes(); + t.column_pieces.push_back(std::move(col)); + } + t.mem.set(bytes); + } + + startMoreThreadsIfNeeded(lock); +} + +void ParquetBlockOutputFormat::reapCompletedRowGroups(std::unique_lock & lock) +{ + while (!row_groups.empty() && row_groups.front().tasks_in_flight == 0 && !is_stopped) + { + RowGroupState & r = row_groups.front(); + + /// Write to the file. + + lock.unlock(); + + if (row_groups_complete.empty()) + writeFileHeader(out); + + std::vector metadata; + for (auto & cols : r.column_chunks) + { + for (ColumnChunk & col : cols) + { + size_t offset = out.count(); + + out.write(col.serialized.data(), col.serialized.size()); + auto m = finalizeColumnChunkAndWriteFooter(offset, std::move(col.state), options, out); + + metadata.push_back(std::move(m)); + } + } + + row_groups_complete.push_back(makeRowGroup(std::move(metadata), r.num_rows)); + + lock.lock(); + + row_groups.pop_front(); } } -void ParquetBlockOutputFormat::finalizeImpl() +void ParquetBlockOutputFormat::startMoreThreadsIfNeeded(const std::unique_lock &) { - if (!file_writer && staging_chunks.empty()) + /// Speculate that all current are already working on tasks. + size_t to_add = std::min(task_queue.size(), format_settings.max_threads - threads_running); + for (size_t i = 0; i < to_add; ++i) { - Block header = materializeBlock(getPort(PortKind::Main).getHeader()); + auto job = [this, thread_group = CurrentThread::getGroup()]() + { + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachFromGroupIfNotDetached();); - consume(Chunk(header.getColumns(), 0)); // this will make staging_chunks non-empty + try + { + setThreadName("ParquetEncoder"); + + threadFunction(); + } + catch (...) + { + std::lock_guard lock(mutex); + background_exception = std::current_exception(); + condvar.notify_all(); + --threads_running; + } + }; + + if (threads_running == 0) + { + /// First thread. We need it to succeed; otherwise we may get stuck. + pool->scheduleOrThrowOnError(job); + ++threads_running; + } + else + { + /// More threads. This may be called from inside the thread pool, so avoid waiting; + /// otherwise it may deadlock. + if (!pool->trySchedule(job)) + break; + } } - - if (!staging_chunks.empty()) - { - consumeStaged(); - staging_chunks.clear(); - staging_rows = 0; - staging_bytes = 0; - } - - auto status = file_writer->Close(); - if (!status.ok()) - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); } -void ParquetBlockOutputFormat::resetFormatterImpl() +void ParquetBlockOutputFormat::threadFunction() { - file_writer.reset(); + std::unique_lock lock(mutex); + + while (true) + { + if (task_queue.empty() || is_stopped) + { + /// The check and the decrement need to be in the same critical section, to make sure + /// we never get stuck with tasks but no threads. + --threads_running; + return; + } + + auto task = std::move(task_queue.front()); + task_queue.pop_front(); + + if (task.column_type) + { + lock.unlock(); + + IColumn::MutablePtr concatenated = IColumn::mutate(std::move(task.column_pieces[0])); + for (size_t i = 1; i < task.column_pieces.size(); ++i) + { + auto & c = task.column_pieces[i]; + concatenated->insertRangeFrom(*c, 0, c->size()); + c.reset(); + } + task.column_pieces.clear(); + + std::vector subcolumns; + prepareColumnForWrite( + std::move(concatenated), task.column_type, task.column_name, options, &subcolumns); + + lock.lock(); + + for (size_t i = 0; i < subcolumns.size(); ++i) + { + task.row_group->column_chunks[task.column_idx].emplace_back(this); + task.row_group->tasks_in_flight += 1; + + auto & t = task_queue.emplace_back(task.row_group, task.column_idx, this); + t.subcolumn_idx = i; + t.state = std::move(subcolumns[i]); + t.mem.set(t.state.allocatedBytes()); + } + + startMoreThreadsIfNeeded(lock); + } + else + { + lock.unlock(); + + PODArray serialized; + { + WriteBufferFromVector buf(serialized); + writeColumnChunkBody(task.state, options, buf); + } + + lock.lock(); + + auto & c = task.row_group->column_chunks[task.column_idx][task.subcolumn_idx]; + c.state = std::move(task.state); + c.serialized = std::move(serialized); + c.mem.set(c.serialized.size() + c.state.allocatedBytes()); + } + + --task.row_group->tasks_in_flight; + + condvar.notify_all(); + } } void registerOutputFormatParquet(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 482c778bc52..4c73de007fe 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -2,8 +2,11 @@ #include "config.h" #if USE_PARQUET -# include -# include + +#include +#include +#include +#include namespace arrow { @@ -28,25 +31,128 @@ class ParquetBlockOutputFormat : public IOutputFormat { public: ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + ~ParquetBlockOutputFormat() override; String getName() const override { return "ParquetBlockOutputFormat"; } String getContentType() const override { return "application/octet-stream"; } private: - void consumeStaged(); + struct MemoryToken + { + ParquetBlockOutputFormat * parent; + size_t bytes = 0; + + explicit MemoryToken(ParquetBlockOutputFormat * p, size_t b = 0) : parent(p) + { + set(b); + } + + MemoryToken(MemoryToken && t) + : parent(std::exchange(t.parent, nullptr)), bytes(std::exchange(t.bytes, 0)) {} + + MemoryToken & operator=(MemoryToken && t) + { + parent = std::exchange(t.parent, nullptr); + bytes = std::exchange(t.bytes, 0); + return *this; + } + + ~MemoryToken() + { + set(0); + } + + void set(size_t new_size) + { + if (new_size == bytes) + return; + parent->bytes_in_flight += new_size - bytes; // overflow is fine + bytes = new_size; + } + }; + + struct ColumnChunk + { + Parquet::ColumnChunkWriteState state; + PODArray serialized; + + MemoryToken mem; + + ColumnChunk(ParquetBlockOutputFormat * p) : mem(p) {} + }; + + struct RowGroupState + { + size_t tasks_in_flight = 0; + std::vector> column_chunks; + size_t num_rows = 0; + }; + + struct Task + { + RowGroupState * row_group; + size_t column_idx; + size_t subcolumn_idx = 0; + + MemoryToken mem; + + /// If not null, we need to call prepareColumnForWrite(). + /// Otherwise we need to call writeColumnChunkBody(). + DataTypePtr column_type; + std::string column_name; + std::vector column_pieces; + + Parquet::ColumnChunkWriteState state; + + Task(RowGroupState * rg, size_t ci, ParquetBlockOutputFormat * p) + : row_group(rg), column_idx(ci), mem(p) {} + }; + void consume(Chunk) override; void finalizeImpl() override; void resetFormatterImpl() override; + void onCancel() override; + void writeRowGroup(std::vector chunks); + void writeUsingArrow(std::vector chunks); + void writeRowGroupInOneThread(Chunk chunk); + void writeRowGroupInParallel(std::vector chunks); + + void threadFunction(); + void startMoreThreadsIfNeeded(const std::unique_lock & lock); + + /// Called in single-threaded fashion. Writes to the file. + void reapCompletedRowGroups(std::unique_lock & lock); + + const FormatSettings format_settings; + + /// Chunks to squash together to form a row group. std::vector staging_chunks; size_t staging_rows = 0; size_t staging_bytes = 0; - const FormatSettings format_settings; - std::unique_ptr file_writer; std::unique_ptr ch_column_to_arrow_column; + + Parquet::WriteOptions options; + Parquet::SchemaElements schema; + std::vector row_groups_complete; + + + std::mutex mutex; + std::condition_variable condvar; // wakes up consume() + std::unique_ptr pool; + + std::atomic_bool is_stopped{false}; + std::exception_ptr background_exception = nullptr; + + /// Invariant: if there's at least one task then there's at least one thread. + size_t threads_running = 0; + std::atomic bytes_in_flight{0}; + + std::deque task_queue; + std::deque row_groups; }; } diff --git a/tests/queries/0_stateless/02735_parquet_encoder.reference b/tests/queries/0_stateless/02735_parquet_encoder.reference new file mode 100644 index 00000000000..c7d79392d85 --- /dev/null +++ b/tests/queries/0_stateless/02735_parquet_encoder.reference @@ -0,0 +1,55 @@ +u8 Nullable(UInt8) +u16 Nullable(UInt16) +u32 Nullable(UInt32) +u64 Nullable(UInt64) +i8 Nullable(Int8) +i16 Nullable(Int16) +i32 Nullable(Int32) +i64 Nullable(Int64) +date Nullable(UInt16) +date32 Nullable(Date32) +datetime Nullable(UInt32) +datetime64 Nullable(DateTime64(3, \'UTC\')) +enum8 Nullable(Int8) +enum16 Nullable(Int16) +float32 Nullable(Float32) +float64 Nullable(Float64) +str Nullable(String) +fstr Nullable(FixedString(12)) +u128 Nullable(FixedString(16)) +u256 Nullable(FixedString(32)) +i128 Nullable(FixedString(16)) +i256 Nullable(FixedString(32)) +decimal32 Nullable(Decimal(9, 3)) +decimal64 Nullable(Decimal(18, 10)) +decimal128 Nullable(Decimal(38, 20)) +decimal256 Nullable(Decimal(76, 40)) +ipv4 Nullable(UInt32) +ipv6 Nullable(FixedString(16)) +0 +0 +0 +0 +1 2 1 +1 2 2 +1 3 3 +1 1000000 1 +3914219105369203805 +4 1000000 1 +(1000000,0,NULL,'100','299') +(1000000,0,NULL,'0','-1294970296') +(1000000,0,NULL,'-2147483296','2147481000') +(100000,900000,NULL,'100009','999999') +[(2,0,NULL,'','[]')] +1 1 +0 1 +16159458007063698496 +16159458007063698496 +BYTE_ARRAY String +FIXED_LEN_BYTE_ARRAY None +BYTE_ARRAY None +BYTE_ARRAY None +BYTE_ARRAY String +never gonna +give you +up diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql new file mode 100644 index 00000000000..d8d52a13218 --- /dev/null +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -0,0 +1,168 @@ +-- Tags: no-fasttest + +set output_format_parquet_use_custom_encoder = 1; +set output_format_parquet_row_group_size = 1000; +set output_format_parquet_data_page_size = 800; +set output_format_parquet_batch_size = 100; +set output_format_parquet_row_group_size_bytes = 1000000000; +set engine_file_truncate_on_insert=1; + +-- Write random data to parquet file, then read from it and check that it matches what we wrote. +-- Do this for all kinds of data types: primitive, Nullable(primitive), Array(primitive), +-- Array(Nullable(primitive)), Array(Array(primitive)), Map(primitive, primitive), etc. + +drop table if exists basic_types_02735; +create temporary table basic_types_02735 as select * from generateRandom(' + u8 UInt8, + u16 UInt16, + u32 UInt32, + u64 UInt64, + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64, + date Date, + date32 Date32, + datetime DateTime, + datetime64 DateTime64, + enum8 Enum8(''x'' = 1, ''y'' = 2, ''z'' = 3), + enum16 Enum16(''xx'' = 1000, ''yy'' = 2000, ''zz'' = 3000), + float32 Float32, + float64 Float64, + str String, + fstr FixedString(12), + u128 UInt128, + u256 UInt256, + i128 Int128, + i256 Int256, + decimal32 Decimal32(3), + decimal64 Decimal64(10), + decimal128 Decimal128(20), + decimal256 Decimal256(40), + ipv4 IPv4, + ipv6 IPv6') limit 10101; +insert into function file(basic_types_02735.parquet) select * from basic_types_02735; +desc file(basic_types_02735.parquet); +select (select sum(cityHash64(*)) from basic_types_02735) - (select sum(cityHash64(*)) from file(basic_types_02735.parquet)); +drop table basic_types_02735; + + +drop table if exists nullables_02735; +create temporary table nullables_02735 as select * from generateRandom(' + u16 Nullable(UInt16), + i64 Nullable(Int64), + datetime64 Nullable(DateTime64), + enum8 Nullable(Enum8(''x'' = 1, ''y'' = 2, ''z'' = 3)), + float64 Nullable(Float64), + str Nullable(String), + fstr Nullable(FixedString(12)), + i256 Nullable(Int256), + decimal256 Nullable(Decimal256(40)), + ipv6 Nullable(IPv6)') limit 10000; +insert into function file(nullables_02735.parquet) select * from nullables_02735; +select (select sum(cityHash64(*)) from nullables_02735) - (select sum(cityHash64(*)) from file(nullables_02735.parquet)); +drop table nullables_02735; + + +-- TODO: When cityHash64() fully supports Nullable: https://github.com/ClickHouse/ClickHouse/pull/48625 +-- the next two blocks can be simplified: arrays_out_02735 intermediate table is not needed, +-- a.csv and b.csv are not needed. + +drop table if exists arrays_02735; +drop table if exists arrays_out_02735; +create table arrays_02735 engine = Memory as select * from generateRandom(' + u32 Array(UInt32), + i8 Array(Int8), + datetime Array(DateTime), + enum16 Array(Enum16(''xx'' = 1000, ''yy'' = 2000, ''zz'' = 3000)), + float32 Array(Float32), + str Array(String), + fstr Array(FixedString(12)), + u128 Array(UInt128), + decimal64 Array(Decimal64(10)), + ipv4 Array(IPv4), + msi Map(String, Int16), + tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 10000; +insert into function file(arrays_02735.parquet) select * from arrays_02735; +create temporary table arrays_out_02735 as arrays_02735; +insert into arrays_out_02735 select * from file(arrays_02735.parquet); +select (select sum(cityHash64(*)) from arrays_02735) - (select sum(cityHash64(*)) from arrays_out_02735); +--select (select sum(cityHash64(*)) from arrays_02735) - +-- (select sum(cityHash64(u32, i8, datetime, enum16, float32, str, fstr, arrayMap(x->reinterpret(x, 'UInt128'), u128), decimal64, ipv4, msi, tup)) from file(arrays_02735.parquet)); +drop table arrays_02735; +drop table arrays_out_02735; + + +drop table if exists madness_02735; +create temporary table madness_02735 as select * from generateRandom(' + aa Array(Array(UInt32)), + aaa Array(Array(Array(UInt32))), + an Array(Nullable(String)), + aan Array(Array(Nullable(FixedString(10)))), + l LowCardinality(String), + ln LowCardinality(Nullable(FixedString(11))), + al Array(LowCardinality(UInt128)), + aaln Array(Array(LowCardinality(Nullable(String)))), + mln Map(LowCardinality(String), Nullable(Int8)), + t Tuple(Map(FixedString(5), Tuple(Array(UInt16), Nullable(UInt16), Array(Tuple(Int8, Decimal64(10))))), Tuple(kitchen UInt64, sink String)), + n Nested(hello UInt64, world Tuple(first String, second FixedString(1))) + ') limit 10000; +insert into function file(madness_02735.parquet) select * from madness_02735; +insert into function file(a.csv) select * from madness_02735 order by tuple(*); +insert into function file(b.csv) select aa, aaa, an, aan, l, ln, arrayMap(x->reinterpret(x, 'UInt128'), al) as al_, aaln, mln, t, n.hello, n.world from file(madness_02735.parquet) order by tuple(aa, aaa, an, aan, l, ln, al_, aaln, mln, t, n.hello, n.world); +select (select sum(cityHash64(*)) from file(a.csv, LineAsString)) - (select sum(cityHash64(*)) from file(b.csv, LineAsString)); +--select (select sum(cityHash64(*)) from madness_02735) - +-- (select sum(cityHash64(aa, aaa, an, aan, l, ln, map(x->reinterpret(x, 'UInt128'), al), aaln, mln, t, n.hello, n.world)) from file(madness_02735.parquet)); +drop table madness_02735; + + +-- Merging input blocks into bigger row groups. +insert into function file(squash_02735.parquet) select '012345' union all select '543210' settings max_block_size = 1; +select num_columns, num_rows, num_row_groups from file(squash_02735.parquet, ParquetMetadata); + +-- Row group size limit in bytes. +insert into function file(row_group_bytes_02735.parquet) select '012345' union all select '543210' settings max_block_size = 1, output_format_parquet_row_group_size_bytes = 5; +select num_columns, num_rows, num_row_groups from file(row_group_bytes_02735.parquet, ParquetMetadata); + +-- Row group size limit in rows. +insert into function file(tiny_row_groups_02735.parquet) select * from numbers(3) settings output_format_parquet_row_group_size = 1; +select num_columns, num_rows, num_row_groups from file(tiny_row_groups_02735.parquet, ParquetMetadata); + +-- 1M unique 8-byte values should exceed dictionary_size_limit (1 MB). +insert into function file(big_column_chunk_02735.parquet) select number from numbers(1000000) settings output_format_parquet_row_group_size = 1000000; +select num_columns, num_rows, num_row_groups from file(big_column_chunk_02735.parquet, ParquetMetadata); +select sum(cityHash64(number)) from file(big_column_chunk_02735.parquet); + +-- Check statistics: signed vs unsigned, null count. Use enough rows to produce multiple pages. +insert into function file(statistics_02735.parquet) select 100 + number%200 as a, toUInt32(number * 3000) as u, toInt32(number * 3000) as i, if(number % 10 == 9, toString(number), null) as s from numbers(1000000) settings output_format_parquet_row_group_size = 1000000; +select num_columns, num_rows, num_row_groups from file(statistics_02735.parquet, ParquetMetadata); +select tupleElement(c, 'statistics') from file(statistics_02735.parquet, ParquetMetadata) array join tupleElement(row_groups[1], 'columns') as c; + +-- Statistics string length limit (max_statistics_size). +insert into function file(long_string_02735.parquet) select toString(range(number * 2000)) from numbers(2); +select tupleElement(tupleElement(row_groups[1], 'columns'), 'statistics') from file(long_string_02735.parquet, ParquetMetadata); + +-- Compression setting. +insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaaaaa', toString(number)) as s from numbers(1000) settings output_format_parquet_row_group_size = 10000, output_format_parquet_compression_method='zstd'; +select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); +insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaaaaa', toString(number)) as s from numbers(1000) settings output_format_parquet_row_group_size = 10000, output_format_parquet_compression_method='none'; +select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); + +-- Single-threaded encoding and Arrow encoder. +drop table if exists other_encoders_02735; +create temporary table other_encoders_02735 as select number, number*2 from numbers(10000); +insert into function file(single_thread_02735.parquet) select * from other_encoders_02735 settings max_threads = 1; +select sum(cityHash64(*)) from file(single_thread_02735.parquet); +insert into function file(arrow_02735.parquet) select * from other_encoders_02735 settings output_format_parquet_use_custom_encoder = 0; +select sum(cityHash64(*)) from file(arrow_02735.parquet); + +-- String -> binary vs string; FixedString -> fixed-length-binary vs binary vs string. +insert into function file(strings1_02735.parquet) select 'never', toFixedString('gonna', 5) settings output_format_parquet_string_as_string = 1, output_format_parquet_fixed_string_as_fixed_byte_array = 1; +select columns.5, columns.6 from file(strings1_02735.parquet, ParquetMetadata) array join columns; +insert into function file(strings2_02735.parquet) select 'give', toFixedString('you', 3) settings output_format_parquet_string_as_string = 0, output_format_parquet_fixed_string_as_fixed_byte_array = 0; +select columns.5, columns.6 from file(strings2_02735.parquet, ParquetMetadata) array join columns; +insert into function file(strings3_02735.parquet) select toFixedString('up', 2) settings output_format_parquet_string_as_string = 1, output_format_parquet_fixed_string_as_fixed_byte_array = 0; +select columns.5, columns.6 from file(strings3_02735.parquet, ParquetMetadata) array join columns; +select * from file(strings1_02735.parquet); +select * from file(strings2_02735.parquet); +select * from file(strings3_02735.parquet); From db5cb960508fc20ff7127aa092b89e6002c9f503 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 17 May 2023 01:56:00 +0000 Subject: [PATCH 731/871] Start over when falling back to non-dictionary encoding --- src/Processors/Formats/Impl/Parquet/Write.cpp | 41 +++++++++++-------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index a29bb81f8dc..ba67f075a0d 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -421,10 +421,7 @@ void writeColumnImpl( typename Converter::Statistics page_statistics; typename Converter::Statistics total_statistics; - /// We start with dictionary encoding, then switch to `encoding` (non-dictionary) if the - /// dictionary gets too big. That's how arrow does it too. - bool initially_used_dictionary = options.use_dictionary_encoding; - bool currently_using_dictionary = initially_used_dictionary; + bool use_dictionary = options.use_dictionary_encoding; std::optional fixed_string_descr; if constexpr (std::is_same::value) @@ -441,12 +438,11 @@ void writeColumnImpl( /// Alternatively, we could avoid using arrow's dictionary encoding code and leverage /// ColumnLowCardinality instead. It would work basically the same way as what this function /// currently does: add values to the ColumnRowCardinality (instead of `encoder`) in batches, - /// checking dictionary size after each batch; if it gets big, flush the dictionary and the - /// indices and switch to non-dictionary encoding. Feels like it could even be slightly less code. + /// checking dictionary size after each batch. That might be faster. auto encoder = parquet::MakeTypedEncoder( // ignored if using dictionary static_cast(encoding), - currently_using_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); + use_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); struct PageData { @@ -496,7 +492,7 @@ void writeColumnImpl( header.__isset.data_page_header = true; auto & d = header.data_page_header; d.__set_num_values(static_cast(def_count)); - d.__set_encoding(currently_using_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); + d.__set_encoding(use_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); d.__set_definition_level_encoding(parq::Encoding::RLE); d.__set_repetition_level_encoding(parq::Encoding::RLE); /// We could also put checksum in `header.crc`, but apparently no one uses it: @@ -513,7 +509,7 @@ void writeColumnImpl( total_statistics.merge(page_statistics); page_statistics.clear(); - if (currently_using_dictionary) + if (use_dictionary) { dict_encoded_pages.push_back({.header = std::move(header)}); std::swap(dict_encoded_pages.back().data, compressed); @@ -593,13 +589,22 @@ void writeColumnImpl( next_def_offset += def_count; next_data_offset += data_count; - if (currently_using_dictionary && is_dict_too_big()) + if (use_dictionary && is_dict_too_big()) { /// Fallback to non-dictionary encoding. - flush_page(next_def_offset - def_offset, next_data_offset - data_offset); - flush_dict(); + /// + /// Discard encoded data and start over. + /// This is different from what arrow does: arrow writes out the dictionary-encoded + /// data, then uses non-dictionary encoding for later pages. + /// Starting over seems better: it produces slightly smaller files (I saw 1-4%) in + /// exchange for slight decrease in speed (I saw < 5%). This seems like a good + /// trade because encoding speed is much less important than decoding (as evidenced + /// by arrow not supporting parallel encoding, even though it's easy to support). - currently_using_dictionary = false; + def_offset = 0; + data_offset = 0; + dict_encoded_pages.clear(); + use_dictionary = false; encoder = parquet::MakeTypedEncoder( static_cast(encoding)); break; @@ -614,7 +619,7 @@ void writeColumnImpl( } } - if (currently_using_dictionary) + if (use_dictionary) flush_dict(); chassert(data_offset == s.primitive_column->size()); @@ -630,13 +635,15 @@ void writeColumnImpl( /// Report which encodings we've used. if (s.max_rep > 0 || s.max_def > 0) addToEncodingsUsed(s, parq::Encoding::RLE); // levels - if (!currently_using_dictionary) - addToEncodingsUsed(s, encoding); // non-dictionary encoding - if (initially_used_dictionary) + if (use_dictionary) { addToEncodingsUsed(s, parq::Encoding::PLAIN); // dictionary itself addToEncodingsUsed(s, parq::Encoding::RLE_DICTIONARY); // ids } + else + { + addToEncodingsUsed(s, encoding); + } } } From dfdf5de972b0b8ee37fd0e89cfeaa8c3f5ea79cf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 May 2023 01:28:16 +0000 Subject: [PATCH 732/871] Fixes --- contrib/arrow-cmake/CMakeLists.txt | 5 +- programs/client/Client.cpp | 5 + .../Formats/Impl/Parquet/PrepareForWrite.cpp | 16 ++- src/Processors/Formats/Impl/Parquet/Write.cpp | 120 +++++++++++++++--- src/Processors/Formats/Impl/Parquet/Write.h | 1 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 7 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 18 ++- .../Formats/Impl/ParquetBlockOutputFormat.h | 1 + .../02581_parquet_arrow_orc_compressions.sh | 2 + .../0_stateless/02735_parquet_encoder.sql | 2 +- 10 files changed, 147 insertions(+), 30 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 46b86cb4ddb..e3ea0381595 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -502,9 +502,10 @@ target_include_directories(_parquet SYSTEM BEFORE "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" "${CMAKE_CURRENT_SOURCE_DIR}/cpp/src") target_link_libraries(_parquet - PUBLIC _arrow - PRIVATE + PUBLIC + _arrow ch_contrib::thrift + PRIVATE boost::headers_only boost::regex OpenSSL::Crypto OpenSSL::SSL) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e1a33231592..e73f77819ad 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -812,6 +812,11 @@ bool Client::processWithFuzzing(const String & full_query) } catch (...) { + if (!ast_to_process) + fmt::print(stderr, + "Error while forming new query: {}\n", + getCurrentExceptionMessage(true)); + // Some functions (e.g. protocol parsers) don't throw, but // set last_exception instead, so we'll also do it here for // uniformity. diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index a70b6fcfc81..0700fc8491c 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -295,7 +295,17 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin switch (type->getTypeId()) { - case TypeIndex::UInt8: types(T::INT32, C::UINT_8 , int_type(8 , false)); break; + case TypeIndex::UInt8: + if (isBool(type)) + { + types(T::BOOLEAN); + state.is_bool = true; + } + else + { + types(T::INT32, C::UINT_8 , int_type(8 , false)); + } + break; case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; @@ -588,7 +598,7 @@ SchemaElements convertSchema(const Block & sample, const WriteOptions & options) root.__set_name("schema"); root.__set_num_children(static_cast(sample.columns())); - for (auto & c : sample) + for (const auto & c : sample) prepareColumnForWrite(c.column, c.type, c.name, options, nullptr, &schema); return schema; @@ -598,7 +608,7 @@ void prepareColumnForWrite( ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema) { - if (column->size() == 0 && out_columns_to_write != nullptr) + if (column->empty() && out_columns_to_write != nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column passed to Parquet encoder"); ColumnChunkWriteStates states; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index ba67f075a0d..9664d173f29 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -15,6 +15,10 @@ #include #include "config_version.h" +#if USE_SNAPPY +#include +#endif + namespace DB::ErrorCodes { extern const int CANNOT_COMPRESS; @@ -68,7 +72,7 @@ struct StatisticsNumeric } }; -struct StatisticsFixedString +struct StatisticsFixedStringRef { size_t fixed_string_size = UINT64_MAX; const uint8_t * min = nullptr; @@ -81,7 +85,7 @@ struct StatisticsFixedString addMax(a.ptr); } - void merge(const StatisticsFixedString & s) + void merge(const StatisticsFixedStringRef & s) { chassert(fixed_string_size == UINT64_MAX || fixed_string_size == s.fixed_string_size); fixed_string_size = s.fixed_string_size; @@ -93,7 +97,7 @@ struct StatisticsFixedString void clear() { min = max = nullptr; } - parq::Statistics get(const WriteOptions & options) + parq::Statistics get(const WriteOptions & options) const { parq::Statistics s; if (min == nullptr || fixed_string_size > options.max_statistics_size) @@ -115,7 +119,54 @@ struct StatisticsFixedString } }; -struct StatisticsString +template +struct StatisticsFixedStringCopy +{ + bool empty = true; + std::array min {}; + std::array max {}; + + void add(parquet::FixedLenByteArray a) + { + addMin(a.ptr); + addMax(a.ptr); + empty = false; + } + + void merge(const StatisticsFixedStringCopy & s) + { + if (s.empty) + return; + addMin(&s.min[0]); + addMax(&s.max[0]); + empty = false; + } + + void clear() { empty = true; } + + parq::Statistics get(const WriteOptions &) const + { + parq::Statistics s; + if (empty) + return s; + s.__set_min_value(std::string(reinterpret_cast(min.data()), S)); + s.__set_max_value(std::string(reinterpret_cast(max.data()), S)); + return s; + } + + void addMin(const uint8_t * p) + { + if (empty || memcmp(p, min.data(), S) < 0) + memcpy(min.data(), p, S); + } + void addMax(const uint8_t * p) + { + if (empty || memcmp(p, max.data(), S) > 0) + memcpy(max.data(), p, S); + } +}; + +struct StatisticsStringRef { parquet::ByteArray min; parquet::ByteArray max; @@ -126,7 +177,7 @@ struct StatisticsString addMax(x); } - void merge(const StatisticsString & s) + void merge(const StatisticsStringRef & s) { if (s.min.ptr == nullptr) return; @@ -136,7 +187,7 @@ struct StatisticsString void clear() { *this = {}; } - parq::Statistics get(const WriteOptions & options) + parq::Statistics get(const WriteOptions & options) const { parq::Statistics s; if (min.ptr == nullptr) @@ -197,7 +248,7 @@ struct ConverterNumeric { buf.resize(count); for (size_t i = 0; i < count; ++i) - buf[i] = static_cast(column.getData()[offset + i]); + buf[i] = static_cast(column.getData()[offset + i]); // NOLINT return buf.data(); } } @@ -205,7 +256,7 @@ struct ConverterNumeric struct ConverterString { - using Statistics = StatisticsString; + using Statistics = StatisticsStringRef; const ColumnString & column; PODArray buf; @@ -226,7 +277,7 @@ struct ConverterString struct ConverterFixedString { - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringRef; const ColumnFixedString & column; PODArray buf; @@ -246,7 +297,7 @@ struct ConverterFixedString struct ConverterFixedStringAsString { - using Statistics = StatisticsString; + using Statistics = StatisticsStringRef; const ColumnFixedString & column; PODArray buf; @@ -267,7 +318,7 @@ struct ConverterNumberAsFixedString { /// Calculate min/max statistics for little-endian fixed strings, not numbers, because parquet /// doesn't know it's numbers. - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringCopy; const ColumnVector & column; PODArray buf; @@ -290,7 +341,7 @@ struct ConverterNumberAsFixedString template struct ConverterDecimal { - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringCopy; const ColumnDecimal & column; PODArray data_buf; @@ -348,6 +399,24 @@ PODArray & compress(PODArray & source, PODArray & scratch, Com return scratch; } +#if USE_SNAPPY + case CompressionMethod::Snappy: + { + size_t max_dest_size = snappy::MaxCompressedLength(source.size()); + + if (max_dest_size > std::numeric_limits::max()) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", formatReadableSizeWithBinarySuffix(source.size())); + + scratch.resize(max_dest_size); + + size_t compressed_size; + snappy::RawCompress(source.data(), source.size(), scratch.data(), &compressed_size); + + scratch.resize(static_cast(compressed_size)); + return scratch; + } +#endif + default: { auto dest_buf = std::make_unique>>(scratch); @@ -421,7 +490,7 @@ void writeColumnImpl( typename Converter::Statistics page_statistics; typename Converter::Statistics total_statistics; - bool use_dictionary = options.use_dictionary_encoding; + bool use_dictionary = options.use_dictionary_encoding && !s.is_bool; std::optional fixed_string_descr; if constexpr (std::is_same::value) @@ -431,7 +500,8 @@ void writeColumnImpl( "", parquet::Repetition::REQUIRED, parquet::Type::FIXED_LEN_BYTE_ARRAY, parquet::ConvertedType::NONE, static_cast(converter.fixedStringSize())), 0, 0); - page_statistics.fixed_string_size = converter.fixedStringSize(); + if constexpr (std::is_same::value) + page_statistics.fixed_string_size = converter.fixedStringSize(); } /// Could use an arena here (by passing a custom MemoryPool), to reuse memory across pages. @@ -605,8 +675,16 @@ void writeColumnImpl( data_offset = 0; dict_encoded_pages.clear(); use_dictionary = false; + +#ifndef NDEBUG + /// Arrow's DictEncoderImpl destructor asserts that FlushValues() was called, so we + /// call it even though we don't need its output. + encoder->FlushValues(); +#endif + encoder = parquet::MakeTypedEncoder( - static_cast(encoding)); + static_cast(encoding), /* use_dictionary */ false, + fixed_string_descr ? &*fixed_string_descr : nullptr); break; } @@ -668,7 +746,13 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option ConverterNumeric, parquet::parquet_dtype::c_type>( \ s.primitive_column)) - case TypeIndex::UInt8 : N(UInt8 , Int32Type); break; + case TypeIndex::UInt8: + if (s.is_bool) + writeColumnImpl(s, options, out, + ConverterNumeric, bool, bool>(s.primitive_column)); + else + N(UInt8 , Int32Type); + break; case TypeIndex::UInt16 : N(UInt16, Int32Type); break; case TypeIndex::UInt32 : N(UInt32, Int32Type); break; case TypeIndex::UInt64 : N(UInt64, Int64Type); break; @@ -769,14 +853,14 @@ parq::ColumnChunk finalizeColumnChunkAndWriteFooter( serializeThriftStruct(s.column_chunk, out); - return std::move(s.column_chunk); + return s.column_chunk; } parq::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows) { parq::RowGroup r; r.__set_num_rows(num_rows); - r.__set_columns(std::move(column_chunks)); + r.__set_columns(column_chunks); r.__set_total_compressed_size(0); for (auto & c : r.columns) { diff --git a/src/Processors/Formats/Impl/Parquet/Write.h b/src/Processors/Formats/Impl/Parquet/Write.h index 333a32e191f..9197eae5384 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.h +++ b/src/Processors/Formats/Impl/Parquet/Write.h @@ -42,6 +42,7 @@ struct ColumnChunkWriteState ColumnPtr primitive_column; CompressionMethod compression; // must match what's inside column_chunk + bool is_bool = false; /// Repetition and definition levels. Produced by prepareColumnForWrite(). /// def is empty iff max_def == 0, which means no arrays or nullables. diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3dde8ad6a6c..be9c600f9bd 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -59,7 +59,12 @@ ParquetBlockInputFormat::ParquetBlockInputFormat( pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, max_decoding_threads); } -ParquetBlockInputFormat::~ParquetBlockInputFormat() = default; +ParquetBlockInputFormat::~ParquetBlockInputFormat() +{ + is_stopped = true; + if (pool) + pool->wait(); +} void ParquetBlockInputFormat::initializeIfNeeded() { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 9a2d9072860..fbf8b3a7c87 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -68,9 +68,8 @@ namespace if (method == FormatSettings::ParquetCompression::GZIP) return parquet::Compression::type::GZIP; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported parquet compression method"); } - } ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) @@ -162,7 +161,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) if (staging_rows >= target_rows * 2) { /// Increase row group size slightly (by < 2x) to avoid a small row group at the end. - size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); + size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); size_t row_group_size = (staging_rows - 1) / num_row_groups + 1; // round up Chunk concatenated = std::move(staging_chunks[0]); @@ -222,7 +221,10 @@ void ParquetBlockOutputFormat::finalizeImpl() } if (row_groups_complete.empty()) + { + base_offset = out.count(); writeFileHeader(out); + } writeFileFooter(std::move(row_groups_complete), schema, options, out); } else @@ -349,12 +351,15 @@ void ParquetBlockOutputFormat::writeRowGroupInOneThread(Chunk chunk) options, &columns_to_write); if (row_groups_complete.empty()) + { + base_offset = out.count(); writeFileHeader(out); + } std::vector column_chunks; for (auto & s : columns_to_write) { - size_t offset = out.count(); + size_t offset = out.count() - base_offset; writeColumnChunkBody(s, options, out); auto c = finalizeColumnChunkAndWriteFooter(offset, std::move(s), options, out); column_chunks.push_back(std::move(c)); @@ -413,14 +418,17 @@ void ParquetBlockOutputFormat::reapCompletedRowGroups(std::unique_lock metadata; for (auto & cols : r.column_chunks) { for (ColumnChunk & col : cols) { - size_t offset = out.count(); + size_t offset = out.count() - base_offset; out.write(col.serialized.data(), col.serialized.size()); auto m = finalizeColumnChunkAndWriteFooter(offset, std::move(col.state), options, out); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 4c73de007fe..aededc39dc4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -138,6 +138,7 @@ private: Parquet::WriteOptions options; Parquet::SchemaElements schema; std::vector row_groups_complete; + size_t base_offset = 0; std::mutex mutex; diff --git a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh index 89b5147f026..d00026d516a 100755 --- a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh +++ b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh @@ -5,6 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +set -o pipefail + $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='none'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='lz4'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='snappy'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index d8d52a13218..3701c685120 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 6b8752f2931fed6483d9221b6f5388e302245f31 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 12:19:35 +0200 Subject: [PATCH 733/871] fix error message --- src/Functions/FunctionToDecimalString.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index 68ad978632e..c16a72115d6 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -41,7 +41,7 @@ public: { FunctionArgumentDescriptors mandatory_args = { {"Value", nullptr, nullptr, nullptr}, - {"precision", &isNativeInteger, &isColumnConst, "const Integer [0-77]"} + {"precision", &isNativeInteger, &isColumnConst, "const Integer"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, {}); From 5ee71bd643caf26b9f533dab1e369f9dc306296b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jul 2023 10:26:26 +0000 Subject: [PATCH 734/871] Work around the clang bug --- src/Processors/Formats/Impl/Parquet/Write.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 9664d173f29..47ef0c53ab5 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -651,6 +651,10 @@ void writeColumnImpl( const typename ParquetDType::c_type * converted = converter.getBatch(next_data_offset, data_count); if (options.write_page_statistics || options.write_column_chunk_statistics) +/// Workaround for clang bug: https://github.com/llvm/llvm-project/issues/63630 +#ifdef MEMORY_SANITIZER +#pragma clang loop vectorize(disable) +#endif for (size_t i = 0; i < data_count; ++i) page_statistics.add(converted[i]); From 155b90c780733a7712956982367088d856ec139b Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 25 Jul 2023 13:47:59 +0300 Subject: [PATCH 735/871] oops --- src/Functions/FunctionToDecimalString.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index c16a72115d6..a965e2c2c90 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -19,10 +19,8 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionToDecimalString : public IFunction 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 736/871] 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 1ca2db44b13..cb89a091d68 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 737/871] 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 927ac6a54e5..732dba6c6f1 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 22fec136c132de820c07c32d0508e6c67af51050 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 25 Jul 2023 14:04:55 +0200 Subject: [PATCH 738/871] Create new StorageView with substituted parameters for every SELECT query of a parameterized view --- src/Interpreters/Context.cpp | 18 +++++++- src/Interpreters/InterpreterSelectQuery.cpp | 24 +--------- .../TranslateQualifiedNamesVisitor.cpp | 7 --- .../TranslateQualifiedNamesVisitor.h | 6 +-- src/Interpreters/TreeRewriter.cpp | 21 ++------- src/Interpreters/TreeRewriter.h | 5 +-- src/Storages/SelectQueryInfo.h | 1 - src/Storages/StorageSnapshot.cpp | 8 +--- src/Storages/StorageSnapshot.h | 2 +- src/Storages/StorageView.cpp | 44 ++----------------- src/Storages/StorageView.h | 11 +---- 11 files changed, 33 insertions(+), 114 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cc77e0fe723..123c2ab8f85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -114,7 +114,10 @@ #include #include #include +#include +#include #include +#include #if USE_ROCKSDB #include @@ -1576,8 +1579,21 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) { + auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + StorageView::replaceQueryParametersIfParametrizedView(query,parameterized_view_values); + + ASTCreateQuery create; + create.select = query->as(); + auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, getQueryContext()); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); + res->startup(); function->prefer_subquery_to_function_formatting = true; - return table; + return res; } } auto hash = table_expression->getTreeHash(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d07a6521544..23a879a9426 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -610,27 +610,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; - NameToNameMap parameter_types; if (view) { query_info.is_parameterized_view = view->isParameterizedView(); - /// We need to fetch the parameters set for SELECT ... FROM parameterized_view() before the query is replaced. - /// replaceWithSubquery replaces the function child and adds the subquery in its place. - /// the parameters are children of function child, if function (which corresponds to parametrised view and has - /// parameters in its arguments: `parametrised_view()`) is replaced the parameters are also gone from tree - /// So we need to get the parameters before they are removed from the tree - /// and after query is replaced, we use these parameters to substitute in the parameterized view query - if (query_info.is_parameterized_view) - { - query_info.parameterized_view_values = analyzeFunctionParamValues(query_ptr); - parameter_types = view->getParameterTypes(); - } view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot, view->isParameterizedView()); - if (query_info.is_parameterized_view) - { - view->replaceQueryParametersIfParametrizedView(query_ptr, query_info.parameterized_view_values); - } - } syntax_analyzer_result = TreeRewriter(context).analyzeSelect( @@ -639,10 +622,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( options, joined_tables.tablesWithColumns(), required_result_column_names, - table_join, - query_info.is_parameterized_view, - query_info.parameterized_view_values, - parameter_types); + table_join); query_info.syntax_analyzer_result = syntax_analyzer_result; @@ -793,7 +773,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); } - source_header = storage_snapshot->getSampleBlockForColumns(required_columns, query_info.parameterized_view_values); + source_header = storage_snapshot->getSampleBlockForColumns(required_columns); } /// Calculate structure of the result. diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index aeb912ddfbb..130ce2194fd 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -249,13 +249,6 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt if (first_table || !data.join_using_columns.contains(column.name)) { std::string column_name = column.name; - - /// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - if (!data.parameter_values.empty()) - column_name - = StorageView::replaceQueryParameterWithValue(column_name, data.parameter_values, data.parameter_types); - addIdentifier(columns, table.table, column_name); } } diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index 6c804ad6c90..73e45fc7ea0 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -28,15 +28,11 @@ public: const TablesWithColumns & tables; std::unordered_set join_using_columns; bool has_columns; - NameToNameMap parameter_values; - NameToNameMap parameter_types; - Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true, const NameToNameMap & parameter_values_ = {}, const NameToNameMap & parameter_types_ = {}) + Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true) : source_columns(source_columns_) , tables(tables_) , has_columns(has_columns_) - , parameter_values(parameter_values_) - , parameter_types(parameter_types_) {} bool hasColumn(const String & name) const { return source_columns.count(name); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 65b5d950975..d44d6cc6ac8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -299,11 +299,10 @@ using ReplacePositionalArgumentsVisitor = InDepthNodeVisitor table_join, - bool is_parameterized_view, - const NameToNameMap parameter_values, - const NameToNameMap parameter_types) const + std::shared_ptr table_join) const { auto * select_query = query->as(); if (!select_query) @@ -1201,7 +1197,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } - translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns, parameter_values, parameter_types); + translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); /// Optimizes logical expressions. LogicalExpressionsOptimizer(select_query, tables_with_columns, settings.optimize_min_equality_disjunction_chain_length.value).perform(); @@ -1259,15 +1255,6 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - /// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - /// the replaced column names will be used in the next steps - if (is_parameterized_view) - { - for (auto & column : result.source_columns) - column.name = StorageView::replaceQueryParameterWithValue(column.name, parameter_values, parameter_types); - } - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames(); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index ea16c432d0f..a171133cd08 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -128,10 +128,7 @@ public: const SelectQueryOptions & select_options = {}, const std::vector & tables_with_columns = {}, const Names & required_result_columns = {}, - std::shared_ptr table_join = {}, - bool is_parameterized_view = false, - const NameToNameMap parameter_values = {}, - const NameToNameMap parameter_types = {}) const; + std::shared_ptr table_join = {}) const; private: static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 13d6909fd52..0f75562e0c1 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -258,7 +258,6 @@ struct SelectQueryInfo bool parallel_replicas_disabled = false; bool is_parameterized_view = false; - NameToNameMap parameterized_view_values; // If limit is not 0, that means it's a trivial limit query. UInt64 limit = 0; diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 31770c9a32b..6abca59268f 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -113,7 +113,7 @@ NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, co return *column; } -Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values) const +Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const { Block res; @@ -121,12 +121,6 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, cons for (const auto & column_name : column_names) { std::string substituted_column_name = column_name; - - /// substituted_column_name is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - if (!parameter_values.empty()) - substituted_column_name = StorageView::replaceValueWithQueryParameter(column_name, parameter_values); - auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); if (column && !object_column) diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 946e8a98cf2..a69f9b95955 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -71,7 +71,7 @@ struct StorageSnapshot NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const; /// Block with ordinary + materialized + aliases + virtuals + subcolumns. - Block getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values = {}) const; + Block getSampleBlockForColumns(const Names & column_names) const; ColumnsDescription getDescriptionForColumns(const Names & column_names) const; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 470def7e197..f0f9b9540de 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -107,7 +107,8 @@ StorageView::StorageView( const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, - const String & comment) + const String & comment, + const bool is_parameterized_view_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -123,8 +124,7 @@ StorageView::StorageView( NormalizeSelectWithUnionQueryVisitor::Data data{SetOperationMode::Unspecified}; NormalizeSelectWithUnionQueryVisitor{data}.visit(description.inner_query); - is_parameterized_view = query.isParameterizedView(); - view_parameter_types = analyzeReceiveQueryParamsWithType(description.inner_query); + is_parameterized_view = is_parameterized_view_ || query.isParameterizedView(); storage_metadata.setSelectQuery(description); setInMemoryMetadata(storage_metadata); } @@ -173,7 +173,7 @@ void StorageView::read( query_plan.addStep(std::move(materializing)); /// And also convert to expected structure. - const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names, query_info.parameterized_view_values); + const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names); const auto & header = query_plan.getCurrentDataStream().header; const auto * select_with_union = current_inner_query->as(); @@ -258,42 +258,6 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ child = view_query; } -String StorageView::replaceQueryParameterWithValue(const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types) -{ - std::string name = column_name; - std::string::size_type pos = 0u; - for (const auto & parameter : parameter_values) - { - if ((pos = name.find(parameter.first)) != std::string::npos) - { - auto parameter_datatype_iterator = parameter_types.find(parameter.first); - size_t parameter_end = pos + parameter.first.size(); - if (parameter_datatype_iterator != parameter_types.end() && name.size() >= parameter_end && (name[parameter_end] == ',' || name[parameter_end] == ')')) - { - String parameter_name("_CAST(" + parameter.second + ", '" + parameter_datatype_iterator->second + "')"); - name.replace(pos, parameter.first.size(), parameter_name); - break; - } - } - } - return name; -} - -String StorageView::replaceValueWithQueryParameter(const String & column_name, const NameToNameMap & parameter_values) -{ - String name = column_name; - std::string::size_type pos = 0u; - for (const auto & parameter : parameter_values) - { - if ((pos = name.find("_CAST(" + parameter.second)) != std::string::npos) - { - name = name.substr(0,pos) + parameter.first + ")"; - break; - } - } - return name; -} - ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name) { ASTTableExpression * table_expression = getFirstTableExpression(select_query); diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index bebecb79ec0..b8bf5585c0f 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -15,7 +15,8 @@ public: const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, - const String & comment); + const String & comment, + const bool is_parameterized_view_=false); std::string getName() const override { return "View"; } bool isView() const override { return true; } @@ -44,17 +45,9 @@ public: static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name, const bool parameterized_view); static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name); - static String replaceQueryParameterWithValue (const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types); - static String replaceValueWithQueryParameter (const String & column_name, const NameToNameMap & parameter_values); - - const NameToNameMap & getParameterTypes() const - { - return view_parameter_types; - } protected: bool is_parameterized_view; - NameToNameMap view_parameter_types; }; } From ed5393ef035f37ab3acd193c69333ff7a0084be7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 25 Jul 2023 12:09:13 +0000 Subject: [PATCH 739/871] Stabilize tests --- .../0_stateless/02494_query_cache_query_log.reference | 6 +----- .../queries/0_stateless/02494_query_cache_query_log.sql | 9 ++++++--- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.reference b/tests/queries/0_stateless/02494_query_cache_query_log.reference index 9037909d121..f9429064456 100644 --- a/tests/queries/0_stateless/02494_query_cache_query_log.reference +++ b/tests/queries/0_stateless/02494_query_cache_query_log.reference @@ -1,16 +1,12 @@ -- Run a query with query cache not enabled 124437993 -QueryStart SELECT 124437993; Unknown QueryFinish SELECT 124437993; None -- Run a query with query cache enabled 124437994 -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write -- Run the same query with query cache enabled 124437994 -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown -QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read -- Throw exception with query cache enabled SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; None diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.sql b/tests/queries/0_stateless/02494_query_cache_query_log.sql index 79a8f4cb62b..aedc39c4486 100644 --- a/tests/queries/0_stateless/02494_query_cache_query_log.sql +++ b/tests/queries/0_stateless/02494_query_cache_query_log.sql @@ -17,7 +17,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437993;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; @@ -31,7 +32,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; @@ -45,7 +47,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; From 568afbbec318d5c38c8281f8ef4bd5873fa76c42 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 09:56:29 +0000 Subject: [PATCH 740/871] added field with refcount to system.remote_data_paths table --- src/Disks/IDisk.h | 5 ++- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 8 +++- .../02791_remote_paths_refcount.reference | 28 ++++++++++++ .../02791_remote_paths_refcount.sql | 43 +++++++++++++++++++ 5 files changed, 82 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.reference create mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.sql diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 2b0ca369a96..a2c5e59237f 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -303,10 +303,11 @@ public: std::string local_path; std::string common_prefix_for_objects; StoredObjects objects; + size_t refcount; LocalPathWithObjectStoragePaths( - const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) - : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_, size_t refcount_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)), refcount(refcount_) {} }; virtual void getRemotePathsRecursive(const String &, std::vector &) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 762151b3808..001cff4cefe 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -82,7 +82,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path), metadata_storage->getHardlinkCount(local_path)); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index eb514d3b3f4..820b1cf3823 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -25,6 +25,7 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"size", std::make_shared()}, + {"refcount", std::make_shared()}, {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); @@ -48,6 +49,7 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); MutableColumnPtr col_size = ColumnUInt64::create(); + MutableColumnPtr col_refcount = ColumnUInt64::create(); MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); @@ -65,19 +67,22 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->supportsCache()) cache = FileCacheFactory::instance().getByName(disk->getCacheName()).cache; - for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects, refcount] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { col_disk_name->insert(disk_name); col_base_path->insert(disk->getPath()); + if (cache) col_cache_base_path->insert(cache->getBasePath()); else col_cache_base_path->insertDefault(); + col_local_path->insert(local_path); col_remote_path->insert(object.remote_path); col_size->insert(object.bytes_size); + col_refcount->insert(refcount); col_namespace->insert(common_prefox_for_objects); if (cache) @@ -101,6 +106,7 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); res_columns.emplace_back(std::move(col_size)); + res_columns.emplace_back(std::move(col_refcount)); res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.reference b/tests/queries/0_stateless/02791_remote_paths_refcount.reference new file mode 100644 index 00000000000..56fb1536205 --- /dev/null +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.reference @@ -0,0 +1,28 @@ +0_0_0_0 0 +0_0_0_0_1 1 +1_0_0_0 0 +1_0_0_0_1 1 +0_0_0_0_1 checksums.txt 0 +0_0_0_0_1 columns.txt 1 +0_0_0_0_1 count.txt 1 +0_0_0_0_1 default_compression_codec.txt 1 +0_0_0_0_1 id.bin 1 +0_0_0_0_1 id.cmrk2 1 +0_0_0_0_1 metadata_version.txt 1 +0_0_0_0_1 minmax_id.idx 1 +0_0_0_0_1 partition.dat 1 +0_0_0_0_1 primary.cidx 1 +0_0_0_0_1 v.bin 1 +0_0_0_0_1 v.cmrk2 1 +1_0_0_0_1 checksums.txt 0 +1_0_0_0_1 columns.txt 0 +1_0_0_0_1 count.txt 1 +1_0_0_0_1 default_compression_codec.txt 0 +1_0_0_0_1 id.bin 1 +1_0_0_0_1 id.cmrk2 1 +1_0_0_0_1 metadata_version.txt 0 +1_0_0_0_1 minmax_id.idx 1 +1_0_0_0_1 partition.dat 1 +1_0_0_0_1 primary.cidx 1 +1_0_0_0_1 v.bin 0 +1_0_0_0_1 v.cmrk2 0 diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql new file mode 100644 index 00000000000..e64df599d32 --- /dev/null +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.sql @@ -0,0 +1,43 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_refcount SYNC; + +CREATE TABLE t_refcount (id UInt64, v UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') +ORDER BY id PARTITION BY id % 2 +SETTINGS + storage_policy = 's3_cache', + allow_remote_fs_zero_copy_replication = 1, + min_bytes_for_wide_part = 0, + compress_marks = 1, + compress_primary_key = 1, + ratio_of_defaults_for_sparse_serialization = 1.0; + +INSERT INTO t_refcount VALUES (1, 10), (2, 20); + +SET mutations_sync = 2; +ALTER TABLE t_refcount UPDATE v = v * 10 WHERE id % 2 = 1; + +SELECT name, active FROM system.parts WHERE database = currentDatabase() AND table = 't_refcount' ORDER BY name; + +WITH splitByChar('/', full_path) AS path_parts +SELECT path_parts[-2] AS part_name, path_parts[-1] AS file_name, refcount +FROM +( + SELECT + path || local_path AS full_path, + substring(full_path, 1, length(full_path) - position(reverse(full_path), '/') + 1) AS part_path, + refcount + FROM system.remote_data_paths + WHERE disk_name = 's3_cache' +) AS paths +INNER JOIN +( + SELECT path + FROM system.parts + WHERE database = currentDatabase() AND table = 't_refcount' AND active +) AS parts +ON paths.part_path = parts.path +ORDER BY part_name, file_name; + +DROP TABLE IF EXISTS t_refcount SYNC; From 328d0a5269407eef6899907d6b9869307a56dfa4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 25 Jul 2023 14:50:27 +0200 Subject: [PATCH 741/871] 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 e6431927805..9e4a63f6ba9 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 20b6a6c977f..d971e4ec658 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 test_shutdown_and_wait(start_cluster): 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 742/871] 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 d971e4ec658..d1373d44d0f 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 test_shutdown_and_wait(start_cluster): 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 743/871] 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 dae2c7dd1d3..6ceb9b5849e 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 a8ace213075..92be30b101a 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 744/871] 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 8b2943c2b73..389d430622d 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 multiprocessing.dummy import Pool 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 c21e04af8db..1fd09f733f0 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 @@ class MySQLConnection: 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 @@ class MySQLConnection: 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 @@ class MySQLConnection: 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 745/871] 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 06a9b62d9de..6179c70ca57 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 6529ff889b0..3ad5abd03ae 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 fd446a8efe8..a1d8a9c90f3 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 746/871] 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 e02fe788653..561b856841b 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 747/871] 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 d3b3d4b545f..ed56edd7503 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 8e33f482ac1..7ba91e48085 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -134,6 +134,7 @@ private: 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 748/871] 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 ee6dcd05d9a..0527ec67e6e 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -114,18 +114,18 @@ public: { 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 @@ public: 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 @@ public: 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 749/871] 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 7533a99107b..00000000000 --- 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 4992d524e5e..00000000000 --- 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 dd75b0f586d..a6da2a66615 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 2ead00cafb4..8bebef5fb00 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 8402165b62b..fc3ea3a13ca 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 7445068207a..00000000000 --- 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 94528ccf2ee..00000000000 --- 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 e69de29bb2d..00000000000 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 e38e57f5cee..00000000000 --- 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 c341df1949e96f938620f22d9fa064514611046b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 24 Jul 2023 01:14:50 +0300 Subject: [PATCH 750/871] draft with retry ConnectionResetException --- src/IO/S3/Client.cpp | 86 ++++++++++++++++--- src/IO/S3/Client.h | 4 + .../test_checking_s3_blobs_paranoid/test.py | 6 +- 3 files changed, 80 insertions(+), 16 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7e20b1a9e8f..94a7b5166da 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -13,6 +13,8 @@ #include #include +#include + #include #include #include @@ -23,6 +25,15 @@ #include +namespace ProfileEvents +{ + extern const Event S3WriteRequestsErrors; + extern const Event S3ReadRequestsErrors; + + extern const Event DiskS3WriteRequestsErrors; + extern const Event DiskS3ReadRequestsErrors; +} + namespace DB { @@ -346,12 +357,12 @@ Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) c Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const { - return doRequest(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); } Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const { - return doRequest(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); } Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const @@ -361,19 +372,19 @@ Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) cons Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const { - return doRequest( + return doRequestWithRetryNetworkErrors( request, [this](const Model::AbortMultipartUploadRequest & req) { return AbortMultipartUpload(req); }); } Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const { - return doRequest( + return doRequestWithRetryNetworkErrors( request, [this](const Model::CreateMultipartUploadRequest & req) { return CreateMultipartUpload(req); }); } Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const { - auto outcome = doRequest( + auto outcome = doRequestWithRetryNetworkErrors( request, [this](const Model::CompleteMultipartUploadRequest & req) { return CompleteMultipartUpload(req); }); if (!outcome.IsSuccess() || provider_type != ProviderType::GCS) @@ -403,32 +414,32 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const Comp Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const { - return doRequest(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); } Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const { - return doRequest(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); } Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const { - return doRequest(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); } Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const { - return doRequest(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); } Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const { - return doRequest(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); } Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const { - return doRequest(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); } Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & request) const @@ -457,7 +468,7 @@ Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & return ComposeObjectOutcome(MakeRequest(req, endpointResolutionOutcome.GetResult(), Aws::Http::HttpMethod::HTTP_PUT)); }; - return doRequest(request, request_fn); + return doRequestWithRetryNetworkErrors(request, request_fn); } template @@ -538,6 +549,57 @@ Client::doRequest(const RequestType & request, RequestFn request_fn) const throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects"); } +template +std::invoke_result_t +Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const +{ + auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) + { + const size_t max_tries = 10; + std::exception_ptr last_exception = nullptr; + for (size_t try_no = 0; try_no < max_tries; ++try_no) + { + try + { + return request_fn_(request_); + } + catch (Poco::Net::ConnectionResetException &) + { + // to do distinguish read/write + if (client_configuration.for_disk_s3) + { + ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); + ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + } + else + { + ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); + } + + tryLogCurrentException(log, "Will retry"); + // to do back off + last_exception = std::current_exception(); + continue; + } + } + + chassert(last_exception); + std::rethrow_exception(last_exception); + +// try +// { +// std::rethrow_exception(last_exception); +// } +// catch (const Poco::Exception & e) +// { +// throw Exception(Exception::CreateFromPocoTag{}, e); +// } + }; + + return doRequest(request, with_retries); +} + bool Client::supportsMultiPartCopy() const { return provider_type != ProviderType::GCS; diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 8904c850553..bb893c21774 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -250,6 +250,10 @@ private: std::invoke_result_t doRequest(const RequestType & request, RequestFn request_fn) const; + template + std::invoke_result_t + doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const; + void updateURIForBucket(const std::string & bucket, S3::URI new_uri) const; std::optional getURIFromError(const Aws::S3::S3Error & error) const; std::optional updateURIForBucketForHead(const std::string & bucket) const; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 5f7a2a6cc7f..6aa149b9c7b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -307,8 +307,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error @@ -386,6 +385,5 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error From 2cc1ac45dd8dda3385e2df1db9ea4fab1789a585 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 18:45:56 +0200 Subject: [PATCH 751/871] update missed error --- src/Functions/FunctionToDecimalString.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index a965e2c2c90..ce52d8b99f6 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -214,7 +214,7 @@ private: ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const { const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); - const auto * precision_col_const = typeid_cast(arguments[1].column.get()); + const auto * precision_col_const = checkAndGetColumnConst>(arguments[1].column.get()); auto result_col = ColumnString::create(); auto * result_col_string = assert_cast(result_col.get()); From 413ec520b3027d9f377aa1929a2855429994ffe3 Mon Sep 17 00:00:00 2001 From: Sanjam Panda Date: Tue, 25 Jul 2023 18:54:27 +0200 Subject: [PATCH 752/871] 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 0527ec67e6e..54c208c5b60 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -113,20 +113,20 @@ public: 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 @@ public: 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 @@ public: 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 4f7bdf308d215478a718e1fe3c157c043702213e Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 18:57:54 +0200 Subject: [PATCH 753/871] add explicit else --- src/Functions/FunctionToDecimalString.h | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index ce52d8b99f6..3dd946203cc 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -38,7 +38,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args = { - {"Value", nullptr, nullptr, nullptr}, + {"Value", &isNumber, nullptr, "Number"}, {"precision", &isNativeInteger, &isColumnConst, "const Integer"} }; @@ -230,8 +230,10 @@ private: { if (precision_col_const) vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); - else + else if (precision_col) vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); @@ -243,8 +245,11 @@ private: { if (precision_col_const) vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); - else + else if (precision_col) vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); + } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); 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 754/871] 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 cd6f65b7b43..198c3265a84 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 755/871] 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 91f6a9bb541..c111ed40a29 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 756/871] 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 d1373d44d0f..67dd03098e9 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 test_shutdown_and_wait(start_cluster): 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 757/871] 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 407fe7d1b01..1075fbaa0f8 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 3928f7ef460f4f4603ceaa065733ac0a7ebc4d16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Jul 2023 09:19:35 +0200 Subject: [PATCH 758/871] Remove peak memory usage from the final message in the client --- src/Common/ProgressIndication.cpp | 3 --- tests/queries/0_stateless/01921_test_progress_bar.py | 1 - 2 files changed, 4 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 960d864660c..5a1929d4ec2 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -101,9 +101,6 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; - auto peak_memory_usage = getMemoryUsage().peak; - if (peak_memory_usage >= 0) - std::cout << "\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 9ce2168e2ae..89eecbc3987 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,3 @@ with client(name="client1>", log=log) as client1: 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) From 2b91a9da78a3cc273a4ed90169da75c92409e06c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 26 Jul 2023 07:51:26 +0000 Subject: [PATCH 759/871] Remove logging from test --- tests/integration/test_backup_s3_storage_class/test.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py index 3c0a0cf7706..2b11f20afc6 100644 --- a/tests/integration/test_backup_s3_storage_class/test.py +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -1,12 +1,6 @@ -import os -import logging - import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", From 991584506f11563d324051236e09bd7c1a3b12d4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 26 Jul 2023 12:42:18 +0400 Subject: [PATCH 760/871] fix a bug when files are finalizated after first write --- src/Storages/MergeTree/GinIndexStore.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index aa0c1fccbc3..91e831270d4 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -243,6 +243,15 @@ void GinIndexStore::finalize() { if (!current_postings.empty()) writeSegment(); + + if (metadata_file_stream) + metadata_file_stream->finalize(); + + if (dict_file_stream) + dict_file_stream->finalize(); + + if (postings_file_stream) + postings_file_stream->finalize(); } void GinIndexStore::initFileStreams() @@ -319,13 +328,8 @@ void GinIndexStore::writeSegment() current_segment.segment_id = getNextSegmentID(); metadata_file_stream->sync(); - metadata_file_stream->finalize(); - dict_file_stream->sync(); - dict_file_stream->finalize(); - postings_file_stream->sync(); - postings_file_stream->finalize(); } GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_) 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 761/871] 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 4926967d2d2..9217fcfddd9 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 b8061309342..73a2965bf44 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 190f3f39f9e..85b376ac39d 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 b5ce4ae0d78..c90b7ebe6f6 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 017d34d40fdd8fe5b03e993b030385ccb20b0ebc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 28 Jun 2023 23:41:51 +0200 Subject: [PATCH 762/871] determine task size by prewhere columns --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 9 ++++++--- src/Storages/MergeTree/MergeTreeReadPool.cpp | 6 ++++-- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 24be644ee55..d14121a97a3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -673,6 +673,7 @@ class IColumn; 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, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \ M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \ + M(Bool, merge_tree_determine_task_size_by_prewhere_columns, true, "Whether to use only prewhere columns size to determine reading task size.", 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. 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) \ diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index fbad7d2f7be..e9e2138d995 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -328,7 +328,10 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf for (const auto & range : part.ranges) part_info->sum_marks += range.end - range.begin; - part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, column_names); + const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info + ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + : column_names; + part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, columns); const auto task_columns = getReadTaskColumns( part_reader_info, @@ -369,9 +372,9 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf } if (prewhere_info) { - for (const auto & columns : task_columns.pre_columns) + for (const auto & cols : task_columns.pre_columns) { - for (const auto & col : columns) + for (const auto & col : cols) { const size_t col_size = part.data_part->getColumnSize(col.name).data_compressed; part_info->estimated_memory_usage_for_single_prefetch += std::min(col_size, settings.prefetch_buffer_size); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 2ab90189f9d..896769d9355 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -73,8 +73,10 @@ MergeTreeReadPool::MergeTreeReadPool( size_t total_marks = 0; for (const auto & part : parts_ranges) { - total_compressed_bytes += getApproxSizeOfPart( - *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info + ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + : column_names_; + total_compressed_bytes += getApproxSizeOfPart(*part.data_part, columns); total_marks += part.getMarksCount(); } From 04180549b094c231a01642cb70fa051bed2f7abb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 13:15:58 +0200 Subject: [PATCH 763/871] 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 c7d4b87694b..36cd32910b5 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 faa38af6533..e66631460f7 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 b4754c6d6fe..f9f9661a7c4 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 From bf301867650194b089b14240d121b1b1eb3b4f6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 13:23:09 +0200 Subject: [PATCH 764/871] upd test --- .../queries/0_stateless/02701_non_parametric_function.reference | 0 tests/queries/0_stateless/02701_non_parametric_function.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.reference create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.sql diff --git a/tests/queries/0_stateless/02701_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql new file mode 100644 index 00000000000..b242bdc72ef --- /dev/null +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -0,0 +1 @@ +SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } From b80a334eb703166ff3c02f42adf0b4e26efd9f2d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 26 Jul 2023 14:08:08 +0200 Subject: [PATCH 765/871] Return zxid in TestKeeper responses --- src/Common/ZooKeeper/IKeeper.h | 2 ++ src/Common/ZooKeeper/TestKeeper.cpp | 21 +++++++++++++++------ src/Common/ZooKeeper/ZooKeeperCommon.h | 1 - 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 5240acc2616..c9325df0d90 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -136,6 +136,8 @@ using ResponseCallback = std::function; struct Response { Error error = Error::ZOK; + int64_t zxid = 0; + Response() = default; Response(const Response &) = default; Response & operator=(const Response &) = default; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 87c87c4fc92..dabc0ae4eef 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -195,6 +195,7 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest std::pair TestKeeperCreateRequest::process(TestKeeper::Container & container, int64_t zxid) const { CreateResponse response; + response.zxid = zxid; Undo undo; if (container.contains(path)) @@ -257,9 +258,10 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai return { std::make_shared(response), undo }; } -std::pair TestKeeperRemoveRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperRemoveRequest::process(TestKeeper::Container & container, int64_t zxid) const { RemoveResponse response; + response.zxid = zxid; Undo undo; auto it = container.find(path); @@ -296,9 +298,10 @@ std::pair TestKeeperRemoveRequest::process(TestKeeper::Contai return { std::make_shared(response), undo }; } -std::pair TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const { ExistsResponse response; + response.zxid = zxid; auto it = container.find(path); if (it != container.end()) @@ -314,9 +317,10 @@ std::pair TestKeeperExistsRequest::process(TestKeeper::Contai return { std::make_shared(response), {} }; } -std::pair TestKeeperGetRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperGetRequest::process(TestKeeper::Container & container, int64_t zxid) const { GetResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) @@ -336,6 +340,7 @@ std::pair TestKeeperGetRequest::process(TestKeeper::Container std::pair TestKeeperSetRequest::process(TestKeeper::Container & container, int64_t zxid) const { SetResponse response; + response.zxid = zxid; Undo undo; auto it = container.find(path); @@ -370,9 +375,10 @@ std::pair TestKeeperSetRequest::process(TestKeeper::Container return { std::make_shared(response), undo }; } -std::pair TestKeeperListRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperListRequest::process(TestKeeper::Container & container, int64_t zxid) const { ListResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) @@ -414,9 +420,10 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe return { std::make_shared(response), {} }; } -std::pair TestKeeperCheckRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperCheckRequest::process(TestKeeper::Container & container, int64_t zxid) const { CheckResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) { @@ -434,10 +441,11 @@ std::pair TestKeeperCheckRequest::process(TestKeeper::Contain return { std::make_shared(response), {} }; } -std::pair TestKeeperSyncRequest::process(TestKeeper::Container & /*container*/, int64_t) const +std::pair TestKeeperSyncRequest::process(TestKeeper::Container & /*container*/, int64_t zxid) const { SyncResponse response; response.path = path; + response.zxid = zxid; return { std::make_shared(std::move(response)), {} }; } @@ -456,6 +464,7 @@ std::pair TestKeeperReconfigRequest::process(TestKeeper::Cont std::pair TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const { MultiResponse response; + response.zxid = zxid; response.responses.reserve(requests.size()); std::vector undo_actions; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 131d19f1ca4..e4b2cc97744 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -28,7 +28,6 @@ using LogElements = std::vector; struct ZooKeeperResponse : virtual Response { XID xid = 0; - int64_t zxid = 0; UInt64 response_created_time_ns = 0; From 42c5c420247af88c9fc7aa7eaad40e38e46918f8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 26 Jul 2023 14:09:19 +0200 Subject: [PATCH 766/871] Set zxid in all multi responses --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index c24eecbafd8..e88d66e5444 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -642,6 +642,8 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) if (op_error == Error::ZOK || op_num == OpNum::Error) dynamic_cast(*response).readImpl(in); + + response->zxid = zxid; } /// Footer. From aa25ce9e3d50d1b590821a1a9d93f0e1edf53e8e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 26 Jul 2023 11:19:20 +0000 Subject: [PATCH 767/871] Follow-up to "Implement support of encrypted elements in configuration file" Cf. PR #50986 - rename XML attribute "encryption_codec" to "encrypted_by" --- docs/en/operations/configuration-files.md | 67 +++++++++++++++---- docs/ru/operations/configuration-files.md | 7 +- src/Common/Config/ConfigProcessor.cpp | 20 +++--- src/Common/examples/encrypt_decrypt.cpp | 4 +- src/Compression/CompressionCodecEncrypted.cpp | 42 ++---------- src/Compression/CompressionCodecEncrypted.h | 4 +- .../test_config_decryption/configs/config.xml | 7 +- .../configs/config.yaml | 6 +- .../configs/config_invalid_chars.xml | 8 ++- .../configs/config_no_encryption_key.xml | 6 +- .../configs/config_subnodes.xml | 6 +- .../configs/config_wrong_method.xml | 7 +- .../test_wrong_settings.py | 2 +- 13 files changed, 110 insertions(+), 76 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index d1d9fa542ab..a19c55673ed 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -67,7 +67,7 @@ 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 `encrypted_by` 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. @@ -75,19 +75,22 @@ Example: ```xml + 00112233445566778899aabbccddeeff + admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + ``` -To get the encrypted value `encrypt_decrypt` example application may be used. +To encrypt a value, you can use the (example) program `encrypt_decrypt`: Example: @@ -138,12 +141,17 @@ Here you can see default config written in YAML: [config.yaml.example](https://g There are some differences between YAML and XML formats in terms of ClickHouse configurations. Here are some tips for writing a configuration in YAML format. -You should use a Scalar node to write a key-value pair: +An XML tag with a text value is represented by a YAML key-value pair ``` yaml key: value ``` -To create a node, containing other nodes you should use a Map: +Corresponding XML: +``` xml +value +``` + +A nested XML node is represented by a YAML map: ``` yaml map_key: key1: val1 @@ -151,7 +159,16 @@ map_key: key3: val3 ``` -To create a list of values or nodes assigned to one tag you should use a Sequence: +Corresponding XML: +``` xml + + val1 + val2 + val3 + +``` + +To create the same XML tag multiple times, use a YAML sequence: ``` yaml seq_key: - val1 @@ -162,8 +179,22 @@ seq_key: key3: val5 ``` -If you want to write an attribute for a Sequence or Map node, you should use a @ prefix before the attribute key. Note, that @ is reserved by YAML standard, so you should also to wrap it into double quotes: +Corresponding XML: +```xml +val1 +val2 + + val3 + + + + val4 + val5 + + +``` +To provide an XML attribute, you can use an attribute key with a `@` prefix. Note that `@` is reserved by YAML standard, so must be wrapped in double quotes: ``` yaml map: "@attr1": value1 @@ -171,16 +202,14 @@ map: key: 123 ``` -From that Map we will get these XML nodes: - +Corresponding XML: ``` xml 123 ``` -You can also set attributes for Sequence: - +It is also possible to use attributes in YAML sequence: ``` yaml seq: - "@attr1": value1 @@ -189,13 +218,25 @@ seq: - abc ``` -So, we can get YAML config equal to this XML one: - +Corresponding XML: ``` xml 123 abc ``` +The aforementioned syntax does not allow to express XML text nodes with XML attributes as YAML. This special case can be achieved using an +`#text` attribute key: +```yaml +map_key: + "@attr1": value1 + "#text": value2 +``` + +Corresponding XML: +```xml +value2 +``` + ## Implementation Details {#implementation-details} For each config file, the server also generates `file-preprocessed.xml` files when starting. These files contain all the completed substitutions and overrides, and they are intended for informational use. If ZooKeeper substitutions were used in the config files but ZooKeeper is not available on the server start, the server loads the configuration from the preprocessed file. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 01a91bd41c6..085761d80c7 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 ## Шифрование {#encryption} -Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec` с именем кодека шифрования как значение к элементу, который надо зашифровать. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encrypted_by` с именем кодека шифрования как значение к элементу, который надо зашифровать. В отличии от аттрибутов `from_zk`, `from_env` и `incl` (или элемента `include`), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. @@ -95,15 +95,18 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ```xml + 00112233445566778899aabbccddeeff + admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + ``` diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index db3c6909b21..a55183782d8 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -192,13 +192,13 @@ static void mergeAttributes(Element & config_element, Element & with_element) std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { - EncryptionMethod method = getEncryptionMethod(codec_name); - CompressionCodecEncrypted codec(method); + EncryptionMethod encryption_method = toEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(encryption_method); 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 encrypted_value(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; @@ -206,8 +206,8 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { - EncryptionMethod method = getEncryptionMethod(codec_name); - CompressionCodecEncrypted codec(method); + EncryptionMethod encryption_method = toEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(encryption_method); Memory<> memory; std::string encrypted_value; @@ -223,7 +223,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()); - std::string decrypted_value = std::string(memory.data(), memory.size()); + std::string decrypted_value(memory.data(), memory.size()); return decrypted_value; } @@ -234,7 +234,7 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) if (node->nodeType() == Node::ELEMENT_NODE) { Element & element = dynamic_cast(*node); - if (element.hasAttribute("encryption_codec")) + if (element.hasAttribute("encrypted_by")) { const NodeListPtr children = element.childNodes(); if (children->length() != 1) @@ -244,8 +244,8 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) if (text_node->nodeType() != Node::TEXT_NODE) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have text node", node->nodeName()); - auto encryption_codec = element.getAttribute("encryption_codec"); - text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); + auto encrypted_by = element.getAttribute("encrypted_by"); + text_node->setNodeValue(decryptValue(encrypted_by, text_node->getNodeValue())); } decryptRecursive(node); } @@ -775,7 +775,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptEncryptedElements(LoadedConfig & loaded_config) { - CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + CompressionCodecEncrypted::Configuration::instance().load(*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/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 503802016cb..c7f949195c8 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 a symmetric encryption codec like AES_128_GCM_SIV or AES_256_GCM_SIV. +/** This 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: @@ -32,7 +32,7 @@ int main(int argc, char ** argv) DB::ConfigProcessor processor(argv[1], false, true); auto loaded_config = processor.loadConfig(); - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + DB::CompressionCodecEncrypted::Configuration::instance().load(*loaded_config.configuration, "encryption_codecs"); if (action == "-e") std::cout << processor.encryptValue(codec_name, value) << std::endl; diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 3f4e35a78a4..5438e02792f 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -31,14 +31,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -EncryptionMethod getEncryptionMethod(const std::string & name) +EncryptionMethod toEncryptionMethod(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); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", name); } namespace @@ -48,34 +48,22 @@ namespace String getMethodName(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return "AES_128_GCM_SIV"; - } else if (Method == AES_256_GCM_SIV) - { return "AES_256_GCM_SIV"; - } else - { return ""; - } } /// Get method code (used for codec, to understand which one we are using) uint8_t getMethodCode(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return static_cast(CompressionMethodByte::AES_128_GCM_SIV); - } else if (Method == AES_256_GCM_SIV) - { return static_cast(CompressionMethodByte::AES_256_GCM_SIV); - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } } // end of namespace @@ -105,17 +93,11 @@ const String empty_nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", actual_nonce_size}; UInt64 methodKeySize(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return 16; - } else if (Method == AES_256_GCM_SIV) - { return 32; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } std::string lastErrorString() @@ -130,17 +112,11 @@ std::string lastErrorString() auto getMethod(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return EVP_aead_aes_128_gcm_siv; - } else if (Method == AES_256_GCM_SIV) - { return EVP_aead_aes_256_gcm_siv; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } /// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag. @@ -206,17 +182,11 @@ size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod m auto getMethod(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return EVP_aes_128_gcm; - } else if (Method == AES_256_GCM_SIV) - { return EVP_aes_256_gcm; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } /// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag. diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index fafcf4af507..7971cbadab7 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,8 +18,8 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; -/// Get method for string name. Throw exception for wrong name. -EncryptionMethod getEncryptionMethod(const std::string & name); +/// Get encryption method for string name. Throw exception for wrong name. +EncryptionMethod toEncryptionMethod(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 diff --git a/tests/integration/test_config_decryption/configs/config.xml b/tests/integration/test_config_decryption/configs/config.xml index 5c274128e39..4b0d3a77659 100644 --- a/tests/integration/test_config_decryption/configs/config.xml +++ b/tests/integration/test_config_decryption/configs/config.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,8 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config.yaml b/tests/integration/test_config_decryption/configs/config.yaml index ab4391be3c5..1b20b65b652 100644 --- a/tests/integration/test_config_decryption/configs/config.yaml +++ b/tests/integration/test_config_decryption/configs/config.yaml @@ -3,9 +3,11 @@ encryption_codecs: key_hex: 00112233445566778899aabbccddeeff aes_256_gcm_siv: key_hex: 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + max_table_size_to_drop: '#text': 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - '@encryption_codec': AES_128_GCM_SIV + '@encrypted_by': AES_128_GCM_SIV + max_partition_size_to_drop: - '@encryption_codec': AES_256_GCM_SIV + '@encrypted_by': AES_256_GCM_SIV '#text': 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml index 49bf51b5bad..53345b897dc 100644 --- a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml +++ b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,9 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + + --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml index 5f7769f7403..830c75f7378 100644 --- a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml @@ -1,3 +1,7 @@ - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + diff --git a/tests/integration/test_config_decryption/configs/config_subnodes.xml b/tests/integration/test_config_decryption/configs/config_subnodes.xml index b0e519ff546..8213270f747 100644 --- a/tests/integration/test_config_decryption/configs/config_subnodes.xml +++ b/tests/integration/test_config_decryption/configs/config_subnodes.xml @@ -1,10 +1,14 @@ + 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 index b452ce6374c..b96c13d5105 100644 --- a/tests/integration/test_config_decryption/configs/config_wrong_method.xml +++ b/tests/integration/test_config_decryption/configs/config_wrong_method.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,8 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + 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 b148f9a051a..c6987d12324 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -15,7 +15,7 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): start_clickhouse( - "configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG" + "configs/config_wrong_method.xml", "Unknown encryption method. Got WRONG" ) From 7d430b803784084a09fd71f98f1a7e8872feb307 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 25 Jul 2023 23:38:03 +0400 Subject: [PATCH 768/871] retry ConnectionResetException --- src/IO/S3/Client.cpp | 79 +++++++++++-------- src/IO/S3/Client.h | 2 +- .../test_checking_s3_blobs_paranoid/test.py | 22 ++++-- 3 files changed, 65 insertions(+), 38 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 94a7b5166da..51c7ee32579 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -357,12 +357,14 @@ Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) c Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); } Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); } Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const @@ -372,19 +374,19 @@ Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) cons Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const { - return doRequestWithRetryNetworkErrors( + return doRequestWithRetryNetworkErrors( request, [this](const Model::AbortMultipartUploadRequest & req) { return AbortMultipartUpload(req); }); } Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const { - return doRequestWithRetryNetworkErrors( + return doRequestWithRetryNetworkErrors( request, [this](const Model::CreateMultipartUploadRequest & req) { return CreateMultipartUpload(req); }); } Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const { - auto outcome = doRequestWithRetryNetworkErrors( + auto outcome = doRequestWithRetryNetworkErrors( request, [this](const Model::CompleteMultipartUploadRequest & req) { return CompleteMultipartUpload(req); }); if (!outcome.IsSuccess() || provider_type != ProviderType::GCS) @@ -414,32 +416,38 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const Comp Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); } Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); } Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); } Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); } Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); } Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); } Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & request) const @@ -468,7 +476,8 @@ Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & return ComposeObjectOutcome(MakeRequest(req, endpointResolutionOutcome.GetResult(), Aws::Http::HttpMethod::HTTP_PUT)); }; - return doRequestWithRetryNetworkErrors(request, request_fn); + return doRequestWithRetryNetworkErrors( + request, request_fn); } template @@ -549,52 +558,60 @@ Client::doRequest(const RequestType & request, RequestFn request_fn) const throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects"); } -template +template std::invoke_result_t Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const { auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) { - const size_t max_tries = 10; + chassert(client_configuration.retryStrategy); + const Int64 max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); std::exception_ptr last_exception = nullptr; - for (size_t try_no = 0; try_no < max_tries; ++try_no) + for (Int64 attempt_no = 0; attempt_no < max_attempts; ++attempt_no) { try { + /// S3 does retries network errors actually. + /// But it is matter when errors occur. + /// This code retries a specific case when + /// network error happens when XML document is being read from the response body. + /// Hence, the response body is a stream, network errors are possible at reading. + /// S3 doesn't retry them. + + /// Not all requests can be retried in that way. + /// Requests that read out response body to build the result are possible to retry. + /// Requests that expose the response stream as an answer are not retried with that code. E.g. GetObject. return request_fn_(request_); } catch (Poco::Net::ConnectionResetException &) { - // to do distinguish read/write - if (client_configuration.for_disk_s3) + + if constexpr (IsReadMethod) { - ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); - ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + if (client_configuration.for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + else + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); } else { - ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); - ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); + if (client_configuration.for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); + else + ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); } tryLogCurrentException(log, "Will retry"); - // to do back off last_exception = std::current_exception(); + + auto error = Aws::Client::AWSError(Aws::Client::CoreErrors::NETWORK_CONNECTION, /*retry*/ true); + client_configuration.retryStrategy->CalculateDelayBeforeNextRetry(error, attempt_no); continue; } } chassert(last_exception); std::rethrow_exception(last_exception); - -// try -// { -// std::rethrow_exception(last_exception); -// } -// catch (const Poco::Exception & e) -// { -// throw Exception(Exception::CreateFromPocoTag{}, e); -// } }; return doRequest(request, with_retries); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index bb893c21774..1b0fdcefe32 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -250,7 +250,7 @@ private: std::invoke_result_t doRequest(const RequestType & request, RequestFn request_fn) const; - template + template std::invoke_result_t doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 6aa149b9c7b..28b0c9beeaa 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -248,7 +248,9 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}" + ) node.query( f""" INSERT INTO @@ -283,7 +285,9 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action="connection_reset_by_peer", action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_1" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}_1" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -307,7 +311,8 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" + in error ), error @@ -325,7 +330,9 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}" + ) node.query( f""" INSERT INTO @@ -361,7 +368,9 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_1" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}_1" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -385,5 +394,6 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" + in error ), error From 9ea479a1a1b77b658f4d8504cb140c09121d44c8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 26 Jul 2023 13:37:33 +0000 Subject: [PATCH 769/871] Analyzer: Support ARRAY JOIN COLUMNS(...) syntax --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 106 ++++++++++-------- .../02833_array_join_columns.reference | 0 .../0_stateless/02833_array_join_columns.sql | 19 ++++ 3 files changed, 78 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/02833_array_join_columns.reference create mode 100644 tests/queries/0_stateless/02833_array_join_columns.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index abf02547ccd..5dbe9e350bf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6477,55 +6477,69 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif resolveExpressionNode(array_join_expression, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - auto result_type = array_join_expression->getResultType(); - bool is_array_type = isArray(result_type); - bool is_map_type = isMap(result_type); - - if (!is_array_type && !is_map_type) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "ARRAY JOIN {} requires expression {} with Array or Map type. Actual {}. In scope {}", - array_join_node_typed.formatASTForErrorMessage(), - array_join_expression->formatASTForErrorMessage(), - result_type->getName(), - scope.scope_node->formatASTForErrorMessage()); - - if (is_map_type) - result_type = assert_cast(*result_type).getNestedType(); - - result_type = assert_cast(*result_type).getNestedType(); - - String array_join_column_name; - - if (!array_join_expression_alias.empty()) + auto process_array_join_expression = [&](QueryTreeNodePtr & expression) { - array_join_column_name = array_join_expression_alias; - } - else if (auto * array_join_expression_inner_column = array_join_expression->as()) + auto result_type = expression->getResultType(); + bool is_array_type = isArray(result_type); + bool is_map_type = isMap(result_type); + + if (!is_array_type && !is_map_type) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN {} requires expression {} with Array or Map type. Actual {}. In scope {}", + array_join_node_typed.formatASTForErrorMessage(), + expression->formatASTForErrorMessage(), + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + if (is_map_type) + result_type = assert_cast(*result_type).getNestedType(); + + result_type = assert_cast(*result_type).getNestedType(); + + String array_join_column_name; + + if (!array_join_expression_alias.empty()) + { + array_join_column_name = array_join_expression_alias; + } + else if (auto * array_join_expression_inner_column = array_join_expression->as()) + { + array_join_column_name = array_join_expression_inner_column->getColumnName(); + } + else if (!identifier_full_name.empty()) + { + array_join_column_name = identifier_full_name; + } + else + { + array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + } + + if (array_join_column_names.contains(array_join_column_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ARRAY JOIN {} multiple columns with name {}. In scope {}", + array_join_node_typed.formatASTForErrorMessage(), + array_join_column_name, + scope.scope_node->formatASTForErrorMessage()); + array_join_column_names.emplace(array_join_column_name); + + NameAndTypePair array_join_column(array_join_column_name, result_type); + auto array_join_column_node = std::make_shared(std::move(array_join_column), expression, array_join_node); + array_join_column_node->setAlias(array_join_expression_alias); + array_join_column_expressions.push_back(std::move(array_join_column_node)); + }; + + // Support ARRAY JOIN COLUMNS(...). COLUMNS trasformer is resolved to list of columns. + if (auto * columns_list = array_join_expression->as()) { - array_join_column_name = array_join_expression_inner_column->getColumnName(); - } - else if (!identifier_full_name.empty()) - { - array_join_column_name = identifier_full_name; + for (auto & array_join_subexpression : columns_list->getNodes()) + process_array_join_expression(array_join_subexpression); } else { - array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); - ++array_join_expressions_counter; + process_array_join_expression(array_join_expression); } - - if (array_join_column_names.contains(array_join_column_name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "ARRAY JOIN {} multiple columns with name {}. In scope {}", - array_join_node_typed.formatASTForErrorMessage(), - array_join_column_name, - scope.scope_node->formatASTForErrorMessage()); - array_join_column_names.emplace(array_join_column_name); - - NameAndTypePair array_join_column(array_join_column_name, result_type); - auto array_join_column_node = std::make_shared(std::move(array_join_column), array_join_expression, array_join_node); - array_join_column_node->setAlias(array_join_expression_alias); - array_join_column_expressions.push_back(std::move(array_join_column_node)); } /** Allow to resolve ARRAY JOIN columns from aliases with types after ARRAY JOIN only after ARRAY JOIN expression list is resolved, because @@ -6537,11 +6551,9 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif * And it is expected that `value_element` inside projection expression list will be resolved as `value_element` expression * with type after ARRAY JOIN. */ - for (size_t i = 0; i < array_join_nodes_size; ++i) + array_join_nodes = std::move(array_join_column_expressions); + for (auto & array_join_column_expression : array_join_nodes) { - auto & array_join_column_expression = array_join_nodes[i]; - array_join_column_expression = std::move(array_join_column_expressions[i]); - auto it = scope.alias_name_to_expression_node.find(array_join_column_expression->getAlias()); if (it != scope.alias_name_to_expression_node.end()) { diff --git a/tests/queries/0_stateless/02833_array_join_columns.reference b/tests/queries/0_stateless/02833_array_join_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02833_array_join_columns.sql b/tests/queries/0_stateless/02833_array_join_columns.sql new file mode 100644 index 00000000000..3f9a33a3959 --- /dev/null +++ b/tests/queries/0_stateless/02833_array_join_columns.sql @@ -0,0 +1,19 @@ +drop table if exists test_array_joins; +drop table if exists v4test_array_joins; + +create table test_array_joins +( + id UInt64 default rowNumberInAllBlocks() + 1, + arr_1 Array(String), + arr_2 Array(String), + arr_3 Array(String), + arr_4 Array(String) +) engine = MergeTree order by id; + +insert into test_array_joins (id,arr_1, arr_2, arr_3, arr_4) +SELECT number,array(randomPrintableASCII(3)),array(randomPrintableASCII(3)),array(randomPrintableASCII(3)),array(randomPrintableASCII(3)) +from numbers(1000); + +create view v4test_array_joins as SELECT * from test_array_joins where id != 10; + +select * from v4test_array_joins array join columns('^arr') where match(arr_4,'a') and id < 100 order by id format Null settings optimize_read_in_order = 0; From dccbe875d247818a17e999ceab5e062537169f80 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 26 Jul 2023 15:37:58 +0200 Subject: [PATCH 770/871] check if storage shutdown before we operate MergeTreeDeduplicationLog --- .../MergeTree/MergeTreeDeduplicationLog.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 09a04f13fc7..53481ab06a0 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -231,6 +233,12 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: return std::make_pair(info, false); } + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); + return {}; + } + chassert(current_writer != nullptr); /// Create new record @@ -261,6 +269,12 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (deduplication_window == 0) return; + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); + return; + } + chassert(current_writer != nullptr); for (auto itr = deduplication_map.begin(); itr != deduplication_map.end(); /* no increment here, we erasing from map */) From 338188ae7f1ccdb399671cbfae584ff79705097a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 26 Jul 2023 14:10:27 +0000 Subject: [PATCH 771/871] fix test --- tests/queries/0_stateless/02791_remote_paths_refcount.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql index e64df599d32..180601738ad 100644 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.sql @@ -2,6 +2,10 @@ DROP TABLE IF EXISTS t_refcount SYNC; +-- Names of parts (on which this test depends) +-- can differ in case of fault injection. +SET insert_keeper_fault_injection_probability = 0.0; + CREATE TABLE t_refcount (id UInt64, v UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') ORDER BY id PARTITION BY id % 2 From 89f2e8cdea8d7f32be735cd86326d1cbed24e158 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 26 Jul 2023 23:02:57 +0800 Subject: [PATCH 772/871] Fix S3 table function does not work for pre-signed URL --- src/TableFunctions/TableFunctionS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c8cc0cddd30..3637b3e9eb2 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -160,7 +160,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context configuration.keys = {configuration.url.key}; if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.uri.getPath(), true); + configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(configuration.url.uri.getPath()).getPath(), true); } void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) From 2479f1352a62adebdc460dbfde4510ad25fc7184 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:11:28 +0200 Subject: [PATCH 773/871] fix deadlocks in StorageTableFunctionProxy --- src/Storages/StorageTableFunction.h | 2 +- .../02828_create_as_table_function_rename.reference | 1 + .../0_stateless/02828_create_as_table_function_rename.sql | 7 +++++++ 3 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.reference create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.sql diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 26cbe1f0233..3939483495e 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -156,7 +156,7 @@ public: void checkTableCanBeDropped() const override {} private: - mutable std::mutex nested_mutex; + mutable std::recursive_mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; const bool add_conversion; diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.reference b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.sql b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql new file mode 100644 index 00000000000..7e24e485fb9 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql @@ -0,0 +1,7 @@ + +drop table if exists t1; +create table t1 as remote('localhost', 'system.one'); +rename table t1 to t2; +select * from t2; +rename table t2 to t1; +drop table t1; From 74f3e76b182411fee1d49e74aa5040cd4a378967 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:15:28 +0200 Subject: [PATCH 774/871] fix build with clang-15 --- src/Common/SystemLogBase.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 3 +++ src/Storages/HDFS/StorageHDFS.h | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index ed5ffd78a7b..3d68fe63227 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -185,7 +185,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) +typename SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 6a68d0f21f7..783c71448fc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -360,6 +360,9 @@ private: struct DownloadInfo { + DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr & file_segment_) + : key(key_), offset(offset_), file_segment(file_segment_) {} + CacheMetadata::Key key; size_t offset; /// We keep weak pointer to file segment diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 74801b68f73..13e46bc1023 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -29,6 +29,8 @@ public: struct PathWithInfo { + PathWithInfo() = default; + PathWithInfo(const String & path_, const std::optional & info_) : path(path_), info(info_) {} String path; std::optional info; }; From b8cac9499d01bd51e4b8a669c7d23104c656dc7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 13:18:09 +0000 Subject: [PATCH 775/871] Add tests to reproduce the problem --- .../0_stateless/02833_local_with_dialect.reference | 2 ++ tests/queries/0_stateless/02833_local_with_dialect.sh | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02833_local_with_dialect.reference create mode 100755 tests/queries/0_stateless/02833_local_with_dialect.sh diff --git a/tests/queries/0_stateless/02833_local_with_dialect.reference b/tests/queries/0_stateless/02833_local_with_dialect.reference new file mode 100644 index 00000000000..dbb67375997 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.reference @@ -0,0 +1,2 @@ +0 +[?2004h[?2004lBye. diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh new file mode 100755 index 00000000000..2a2e1b09459 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +echo "exit" | ${CLICKHOUSE_LOCAL} --query "from s\"SELECT * FROM numbers(1)\"" --dialect prql --interactive From 3a6aaa29c9db0db1bc2875b7323c334148da387f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 15:25:57 +0000 Subject: [PATCH 776/871] Do not load suggestions in case not ClickHouse dialects --- programs/local/LocalServer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..6ac7edaf1d9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -569,7 +569,9 @@ void LocalServer::processConfig() } print_stack_trace = config().getBool("stacktrace", false); - load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false); + const std::string clickhouse_dialect{"clickhouse"}; + load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false) + && config().getString("dialect", clickhouse_dialect) == clickhouse_dialect; auto logging = (config().has("logger.console") || config().has("logger.level") From 0a838dc6d19af963a021aa1910f2144839f21d4a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 18:30:18 +0200 Subject: [PATCH 777/871] Revert "Remove `mmap/mremap/munmap` from Allocator.h" (#52589) --- src/Common/Allocator.cpp | 26 ++- src/Common/Allocator.h | 182 ++++++++++++++---- src/Common/Allocator_fwd.h | 2 +- src/Common/CurrentMetrics.cpp | 2 + src/Common/HashTable/HashTableAllocator.h | 2 +- .../01778_mmap_cache_infra.reference | 2 + 6 files changed, 177 insertions(+), 39 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 769df70d71e..0fb90e5a47e 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,4 +1,26 @@ #include "Allocator.h" -template class Allocator; -template class Allocator; +/** Keep definition of this constant in cpp file; otherwise its value + * is inlined into allocator code making it impossible to override it + * in third-party code. + * + * Note: extern may seem redundant, but is actually needed due to bug in GCC. + * See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html + */ +#ifdef NDEBUG + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 128 * (1ULL << 20); +#else + /** + * In debug build, use small mmap threshold to reproduce more memory + * stomping bugs. Along with ASLR it will hopefully detect more issues than + * ASan. The program may fail due to the limit on number of memory mappings. + * + * Not too small to avoid too quick exhaust of memory mappings. + */ + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384; +#endif + +template class Allocator; +template class Allocator; +template class Allocator; +template class Allocator; diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 1e77e988326..5180fbdaa2d 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -36,26 +36,51 @@ #include +/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS +#ifndef MAP_ANONYMOUS +#define MAP_ANONYMOUS MAP_ANON +#endif + +/** + * Many modern allocators (for example, tcmalloc) do not do a mremap for + * realloc, even in case of large enough chunks of memory. Although this allows + * you to increase performance and reduce memory consumption during realloc. + * To fix this, we do mremap manually if the chunk of memory is large enough. + * The threshold (64 MB) is chosen quite large, since changing the address + * space is very slow, especially in the case of a large number of threads. We + * expect that the set of operations mmap/something to do/mremap can only be + * performed about 1000 times per second. + * + * P.S. This is also required, because tcmalloc can not allocate a chunk of + * memory greater than 16 GB. + * + * P.P.S. Note that MMAP_THRESHOLD symbol is intentionally made weak. It allows + * to override it during linkage when using ClickHouse as a library in + * third-party applications which may already use own allocator doing mmaps + * in the implementation of alloc/realloc. + */ +extern const size_t MMAP_THRESHOLD; + static constexpr size_t MALLOC_MIN_ALIGNMENT = 8; +namespace CurrentMetrics +{ + extern const Metric MMappedAllocs; + extern const Metric MMappedAllocBytes; +} + namespace DB { - namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MUNMAP; + extern const int CANNOT_MREMAP; extern const int LOGICAL_ERROR; } - } -/** Previously there was a code which tried to use manual mmap and mremap (clickhouse_mremap.h) for large allocations/reallocations (64MB+). - * Most modern allocators (including jemalloc) don't use mremap, so the idea was to take advantage from mremap system call for large reallocs. - * Actually jemalloc had support for mremap, but it was intentionally removed from codebase https://github.com/jemalloc/jemalloc/commit/e2deab7a751c8080c2b2cdcfd7b11887332be1bb. - * Our performance tests also shows that without manual mmap/mremap/munmap clickhouse is overall faster for about 1-2% and up to 5-7x for some types of queries. - * That is why we don't do manuall mmap/mremap/munmap here and completely rely on jemalloc for allocations of any size. - */ - /** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena. * Also used in hash tables. * The interface is different from std::allocator @@ -63,8 +88,10 @@ namespace ErrorCodes * - passing the size into the `free` method; * - by the presence of the `alignment` argument; * - the possibility of zeroing memory (used in hash tables); + * - random hint address for mmap + * - mmap_threshold for using mmap less or more */ -template +template class Allocator { public: @@ -82,7 +109,7 @@ public: try { checkSize(size); - freeNoTrack(buf); + freeNoTrack(buf, size); CurrentMemoryTracker::free(size); } catch (...) @@ -105,26 +132,49 @@ public: /// nothing to do. /// BTW, it's not possible to change alignment while doing realloc. } - else if (alignment <= MALLOC_MIN_ALIGNMENT) + else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD + && alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. CurrentMemoryTracker::realloc(old_size, new_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) - { - DB::throwFromErrno( - fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } + DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; if constexpr (clear_memory) if (new_size > old_size) memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); } + else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) + { + /// Resize mmap'd memory region. + CurrentMemoryTracker::realloc(old_size, new_size); + + // On apple and freebsd self-implemented mremap used (common/mremap.h) + buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, + PROT_READ | PROT_WRITE, mmap_flags, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", + ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); + + /// No need for zero-fill, because mmap guarantees it. + } + else if (new_size < MMAP_THRESHOLD) + { + /// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once. + CurrentMemoryTracker::realloc(old_size, new_size); + + void * new_buf = allocNoTrack(new_size, alignment); + memcpy(new_buf, buf, std::min(old_size, new_size)); + freeNoTrack(buf, old_size); + buf = new_buf; + } else { /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. + void * new_buf = alloc(new_size, alignment); memcpy(new_buf, buf, std::min(old_size, new_size)); free(buf, old_size); @@ -142,38 +192,83 @@ protected: static constexpr bool clear_memory = clear_memory_; + // Freshly mmapped pages are copy-on-write references to a global zero page. + // On the first write, a page fault occurs, and an actual writable page is + // allocated. If we are going to use this memory soon, such as when resizing + // hash tables, it makes sense to pre-fault the pages by passing + // MAP_POPULATE to mmap(). This takes some time, but should be faster + // overall than having a hot loop interrupted by page faults. + // It is only supported on Linux. + static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS +#if defined(OS_LINUX) + | (mmap_populate ? MAP_POPULATE : 0) +#endif + ; + private: void * allocNoTrack(size_t size, size_t alignment) { void * buf; - if (alignment <= MALLOC_MIN_ALIGNMENT) - { - if constexpr (clear_memory) - buf = ::calloc(size, 1); - else - buf = ::malloc(size); + size_t mmap_min_alignment = ::getPageSize(); - if (nullptr == buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + if (size >= MMAP_THRESHOLD) + { + if (alignment > mmap_min_alignment) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Too large alignment {}: more than page size when allocating {}.", + ReadableSize(alignment), ReadableSize(size)); + + buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE, + mmap_flags, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + /// No need for zero-fill, because mmap guarantees it. + + CurrentMetrics::add(CurrentMetrics::MMappedAllocs); + CurrentMetrics::add(CurrentMetrics::MMappedAllocBytes, size); } else { - buf = nullptr; - int res = posix_memalign(&buf, alignment, size); + if (alignment <= MALLOC_MIN_ALIGNMENT) + { + if constexpr (clear_memory) + buf = ::calloc(size, 1); + else + buf = ::malloc(size); - if (0 != res) - DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), - DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + if (nullptr == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else + { + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); - if constexpr (clear_memory) - memset(buf, 0, size); + if (0 != res) + DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + + if constexpr (clear_memory) + memset(buf, 0, size); + } } return buf; } - void freeNoTrack(void * buf) + void freeNoTrack(void * buf, size_t size) { - ::free(buf); + if (size >= MMAP_THRESHOLD) + { + if (0 != munmap(buf, size)) + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); + + CurrentMetrics::sub(CurrentMetrics::MMappedAllocs); + CurrentMetrics::sub(CurrentMetrics::MMappedAllocBytes, size); + } + else + { + ::free(buf); + } } void checkSize(size_t size) @@ -182,6 +277,21 @@ private: if (size >= 0x8000000000000000ULL) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); } + +#ifndef NDEBUG + /// In debug builds, request mmap() at random addresses (a kind of ASLR), to + /// reproduce more memory stomping bugs. Note that Linux doesn't do it by + /// default. This may lead to worse TLB performance. + void * getMmapHint() + { + return reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); + } +#else + void * getMmapHint() + { + return nullptr; + } +#endif }; @@ -257,5 +367,7 @@ constexpr size_t allocatorInitialBytes; -extern template class Allocator; +extern template class Allocator; +extern template class Allocator; +extern template class Allocator; +extern template class Allocator; diff --git a/src/Common/Allocator_fwd.h b/src/Common/Allocator_fwd.h index a96bc2a503b..a13a4398654 100644 --- a/src/Common/Allocator_fwd.h +++ b/src/Common/Allocator_fwd.h @@ -3,7 +3,7 @@ * This file provides forward declarations for Allocator. */ -template +template class Allocator; template diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 583b13cf79d..e290fc8ccd3 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -173,6 +173,8 @@ M(PartsInMemory, "In-memory parts.") \ M(MMappedFiles, "Total number of mmapped files.") \ M(MMappedFileBytes, "Sum size of mmapped file regions.") \ + M(MMappedAllocs, "Total number of mmapped allocations") \ + M(MMappedAllocBytes, "Sum bytes of mmapped allocations") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ M(PendingAsyncInsert, "Number of asynchronous inserts that are waiting for flush.") \ M(KafkaConsumers, "Number of active Kafka consumers") \ diff --git a/src/Common/HashTable/HashTableAllocator.h b/src/Common/HashTable/HashTableAllocator.h index 8252265111d..47e3fdfc4b6 100644 --- a/src/Common/HashTable/HashTableAllocator.h +++ b/src/Common/HashTable/HashTableAllocator.h @@ -8,7 +8,7 @@ * table, so it makes sense to pre-fault the pages so that page faults don't * interrupt the resize loop. Set the allocator parameter accordingly. */ -using HashTableAllocator = Allocator; +using HashTableAllocator = Allocator; template using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory; diff --git a/tests/queries/0_stateless/01778_mmap_cache_infra.reference b/tests/queries/0_stateless/01778_mmap_cache_infra.reference index 0e82b277bc1..ed365028ecc 100644 --- a/tests/queries/0_stateless/01778_mmap_cache_infra.reference +++ b/tests/queries/0_stateless/01778_mmap_cache_infra.reference @@ -2,5 +2,7 @@ CreatedReadBufferMMap CreatedReadBufferMMapFailed MMappedFileCacheHits MMappedFileCacheMisses +MMappedAllocBytes +MMappedAllocs MMappedFileBytes MMappedFiles From d89e2e6a27746dbb8febd2990d1ed3c23fcf153b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 26 Jul 2023 19:58:41 +0200 Subject: [PATCH 778/871] Add SYSTEM STOP LISTEN query (#51016) Co-authored-by: Nikita Mikhaylov Co-authored-by: Nikita Mikhaylov --- docs/en/sql-reference/statements/system.md | 26 + programs/keeper/CMakeLists.txt | 1 + programs/server/Server.cpp | 448 +++++++++++------- programs/server/Server.h | 14 +- src/Access/Common/AccessType.h | 1 + src/Interpreters/Context.cpp | 34 ++ src/Interpreters/Context.h | 8 + src/Interpreters/InterpreterSystemQuery.cpp | 19 +- src/Parsers/ASTSystemQuery.cpp | 11 + src/Parsers/ASTSystemQuery.h | 7 +- src/Parsers/ParserSystemQuery.cpp | 36 ++ src/Parsers/examples/CMakeLists.txt | 4 +- src/Server/ServerType.cpp | 138 ++++++ src/Server/ServerType.h | 44 ++ .../test_system_start_stop_listen/__init__.py | 0 .../configs/cluster.xml | 16 + .../test_system_start_stop_listen/test.py | 40 ++ .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 +- 19 files changed, 663 insertions(+), 191 deletions(-) create mode 100644 src/Server/ServerType.cpp create mode 100644 src/Server/ServerType.h create mode 100644 tests/integration/test_system_start_stop_listen/__init__.py create mode 100644 tests/integration/test_system_start_stop_listen/configs/cluster.xml create mode 100644 tests/integration/test_system_start_stop_listen/test.py diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 65a35f03fbe..fb601cd5d35 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -414,3 +414,29 @@ Will do sync syscall. ```sql SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name] ``` + + +### SYSTEM STOP LISTEN + +Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol. + +However, if the corresponding protocol settings were not specified in the clickhouse-server configuration, this command will have no effect. + +```sql +SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol'] +``` + +- If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped. +- If `QUERIES ALL` modifier is specified, all protocols are stopped. +- If `QUERIES DEFAULT` modifier is specified, all default protocols are stopped. +- If `QUERIES CUSTOM` modifier is specified, all custom protocols are stopped. + +### SYSTEM START LISTEN + +Allows new connections to be established on the specified protocols. + +However, if the server on the specified port and protocol was not stopped using the SYSTEM STOP LISTEN command, this command will have no effect. + +```sql +SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol'] +``` diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index abf31a7a499..43a8d84b513 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -65,6 +65,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ServerType.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 960b6574633..dce52ecdb12 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1457,6 +1457,24 @@ try access_control.reload(AccessControl::ReloadMode::USERS_CONFIG_ONLY); }); + global_context->setStopServersCallback([&](const ServerType & server_type) + { + stopServers(servers, server_type); + }); + + global_context->setStartServersCallback([&](const ServerType & server_type) + { + createServers( + config(), + listen_hosts, + listen_try, + server_pool, + async_metrics, + servers, + /* start_servers= */ true, + server_type); + }); + /// Limit on total number of concurrently executed queries. global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries); @@ -1998,7 +2016,8 @@ void Server::createServers( Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers) + bool start_servers, + const ServerType & server_type) { const Settings & settings = global_context->getSettingsRef(); @@ -2012,6 +2031,9 @@ void Server::createServers( for (const auto & protocol : protocols) { + if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + continue; + std::vector hosts; if (config.has("protocols." + protocol + ".host")) hosts.push_back(config.getString("protocols." + protocol + ".host")); @@ -2058,162 +2080,190 @@ void Server::createServers( for (const auto & listen_host : listen_hosts) { - /// HTTP - const char * port_name = "http_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); + const char * port_name; - return ProtocolServerAdapter( - listen_host, - port_name, - "http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); - }); - - /// HTTPS - port_name = "https_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::HTTP)) { + /// HTTP + port_name = "http_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + + return ProtocolServerAdapter( + listen_host, + port_name, + "http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); + }); + } + + if (server_type.shouldStart(ServerType::Type::HTTPS)) + { + /// HTTPS + port_name = "https_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "https://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "https://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); + } - /// TCP - port_name = "tcp_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); + /// TCP + port_name = "tcp_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } - /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt - port_name = "tcp_with_proxy_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp) with PROXY: " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); + /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt + port_name = "tcp_with_proxy_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp) with PROXY: " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } - /// TCP with SSL - port_name = "tcp_port_secure"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "secure native protocol (tcp_secure): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); -#else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); + /// TCP with SSL + port_name = "tcp_port_secure"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + #if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure native protocol (tcp_secure): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + #else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + #endif + }); + } - port_name = "mysql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::MYSQL)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "MySQL compatibility protocol: " + address.toString(), - std::make_unique(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); - }); + port_name = "mysql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "MySQL compatibility protocol: " + address.toString(), + std::make_unique(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } - port_name = "postgresql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "PostgreSQL compatibility protocol: " + address.toString(), - std::make_unique(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); - }); + port_name = "postgresql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "PostgreSQL compatibility protocol: " + address.toString(), + std::make_unique(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } #if USE_GRPC - port_name = "grpc_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::GRPC)) { - Poco::Net::SocketAddress server_address(listen_host, port); - return ProtocolServerAdapter( - listen_host, - port_name, - "gRPC protocol: " + server_address.toString(), - std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); - }); + port_name = "grpc_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::SocketAddress server_address(listen_host, port); + return ProtocolServerAdapter( + listen_host, + port_name, + "gRPC protocol: " + server_address.toString(), + std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); + }); + } #endif - - /// Prometheus (if defined and not setup yet with http_port) - port_name = "prometheus.port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); - }); + /// Prometheus (if defined and not setup yet with http_port) + port_name = "prometheus.port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + }); + } } } @@ -2224,7 +2274,8 @@ void Server::createInterserverServers( Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers) + bool start_servers, + const ServerType & server_type) { const Settings & settings = global_context->getSettingsRef(); @@ -2236,52 +2287,97 @@ void Server::createInterserverServers( /// Now iterate over interserver_listen_hosts for (const auto & interserver_listen_host : interserver_listen_hosts) { - /// Interserver IO HTTP - const char * port_name = "interserver_http_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "replica communication (interserver): http://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); - }); + const char * port_name; - port_name = "interserver_https_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) { + /// Interserver IO HTTP + port_name = "interserver_http_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "replica communication (interserver): http://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params)); + }); + } + + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) + { + port_name = "interserver_https_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "secure replica communication (interserver): https://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "secure replica communication (interserver): https://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), + server_pool, + socket, + http_params)); #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); + } } } +void Server::stopServers( + std::vector & servers, + const ServerType & server_type +) const +{ + Poco::Logger * log = &logger(); + + /// Remove servers once all their connections are closed + auto check_server = [&log](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG(log, "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); + + for (auto & server : servers) + { + if (!server.isStopping()) + { + const std::string server_port_name = server.getPortName(); + + if (server_type.shouldStop(server_port_name)) + server.stop(); + } + } + + std::erase_if(servers, std::bind_front(check_server, "")); +} + void Server::updateServers( Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, diff --git a/programs/server/Server.h b/programs/server/Server.h index d13378dcd65..3f03dd137ef 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -3,8 +3,9 @@ #include #include -#include "Server/HTTP/HTTPContext.h" +#include #include +#include #include /** Server provides three interfaces: @@ -106,7 +107,8 @@ private: Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers = false); + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); void createInterserverServers( Poco::Util::AbstractConfiguration & config, @@ -115,7 +117,8 @@ private: Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers = false); + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); void updateServers( Poco::Util::AbstractConfiguration & config, @@ -123,6 +126,11 @@ private: AsynchronousMetrics & async_metrics, std::vector & servers, std::vector & servers_to_start_before_tables); + + void stopServers( + std::vector & servers, + const ServerType & server_type + ) const; }; } diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 374a1dd04a4..b253a0e13ce 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -187,6 +187,7 @@ enum class AccessType 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) \ M(SYSTEM_FAILPOINT, "SYSTEM ENABLE FAILPOINT, SYSTEM DISABLE FAILPOINT", GLOBAL, SYSTEM) \ + M(SYSTEM_LISTEN, "SYSTEM START LISTEN, SYSTEM STOP LISTEN", GLOBAL, SYSTEM) \ M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \ \ M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cc77e0fe723..f83e524ffb9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -357,6 +358,9 @@ struct ContextSharedPart : boost::noncopyable Context::ConfigReloadCallback config_reload_callback; + Context::StartStopServersCallback start_servers_callback; + Context::StartStopServersCallback stop_servers_callback; + bool is_server_completely_started = false; #if USE_ROCKSDB @@ -3688,6 +3692,36 @@ void Context::reloadConfig() const shared->config_reload_callback(); } +void Context::setStartServersCallback(StartStopServersCallback && callback) +{ + /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. + shared->start_servers_callback = std::move(callback); +} + +void Context::setStopServersCallback(StartStopServersCallback && callback) +{ + /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. + shared->stop_servers_callback = std::move(callback); +} + +void Context::startServers(const ServerType & server_type) const +{ + /// Use mutex if callback may be changed after startup. + if (!shared->start_servers_callback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't start servers because start_servers_callback is not set."); + + shared->start_servers_callback(server_type); +} + +void Context::stopServers(const ServerType & server_type) const +{ + /// Use mutex if callback may be changed after startup. + if (!shared->stop_servers_callback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't stop servers because stop_servers_callback is not set."); + + shared->stop_servers_callback(server_type); +} + void Context::shutdown() { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fa210f04451..75752774d4c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -134,6 +134,7 @@ using StoragePolicyPtr = std::shared_ptr; using StoragePoliciesMap = std::map; class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; +class ServerType; template class MergeTreeBackgroundExecutor; @@ -1057,6 +1058,13 @@ public: void setConfigReloadCallback(ConfigReloadCallback && callback); void reloadConfig() const; + using StartStopServersCallback = std::function; + void setStartServersCallback(StartStopServersCallback && callback); + void setStopServersCallback(StartStopServersCallback && callback); + + void startServers(const ServerType & server_type) const; + void stopServers(const ServerType & server_type) const; + void shutdown(); bool isInternalQuery() const { return is_internal_query; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1bd30e06888..3207da9941a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -556,6 +556,14 @@ BlockIO InterpreterSystemQuery::execute() ); break; } + case Type::STOP_LISTEN: + getContext()->checkAccess(AccessType::SYSTEM_LISTEN); + getContext()->stopServers(query.server_type); + break; + case Type::START_LISTEN: + getContext()->checkAccess(AccessType::SYSTEM_LISTEN); + getContext()->startServers(query.server_type); + break; case Type::FLUSH_ASYNC_INSERT_QUEUE: { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); @@ -567,9 +575,6 @@ BlockIO InterpreterSystemQuery::execute() queue->flushAll(); break; } - case Type::STOP_LISTEN_QUERIES: - case Type::START_LISTEN_QUERIES: - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type); case Type::STOP_THREAD_FUZZER: getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER); ThreadFuzzer::stop(); @@ -1181,8 +1186,12 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_SYNC_FILE_CACHE); break; } - case Type::STOP_LISTEN_QUERIES: - case Type::START_LISTEN_QUERIES: + case Type::STOP_LISTEN: + case Type::START_LISTEN: + { + required_access.emplace_back(AccessType::SYSTEM_LISTEN); + break; + } case Type::STOP_THREAD_FUZZER: case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a91449ff035..754eb825dcc 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -220,6 +220,17 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, { settings.ostr << (settings.hilite ? hilite_none : ""); } + else if (type == Type::START_LISTEN || type == Type::STOP_LISTEN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " " << ServerType::serverTypeToString(server_type.type) + << (settings.hilite ? hilite_none : ""); + + if (server_type.type == ServerType::CUSTOM) + { + settings.ostr << (settings.hilite ? hilite_identifier : "") << " " << backQuoteIfNeed(server_type.custom_name); + } + + } } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index b18f8fc7b07..ebaf357c0ab 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include "config.h" @@ -35,8 +36,8 @@ public: #if USE_AWS_S3 DROP_S3_CLIENT_CACHE, #endif - STOP_LISTEN_QUERIES, - START_LISTEN_QUERIES, + STOP_LISTEN, + START_LISTEN, RESTART_REPLICAS, RESTART_REPLICA, RESTORE_REPLICA, @@ -116,6 +117,8 @@ public: SyncReplicaMode sync_replica_mode = SyncReplicaMode::DEFAULT; + ServerType server_type; + String getID(char) const override { return "SYSTEM query"; } ASTPtr clone() const override diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 48dbe60e241..9aff0e8879e 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -442,6 +442,42 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & break; } + case Type::START_LISTEN: + case Type::STOP_LISTEN: + { + if (!parseQueryWithOnCluster(res, pos, expected)) + return false; + + ServerType::Type current_type = ServerType::Type::END; + std::string current_custom_name; + + for (const auto & type : magic_enum::enum_values()) + { + if (ParserKeyword{ServerType::serverTypeToString(type)}.ignore(pos, expected)) + { + current_type = type; + break; + } + } + + if (current_type == ServerType::Type::END) + return false; + + if (current_type == ServerType::CUSTOM) + { + ASTPtr ast; + + if (!ParserStringLiteral{}.parse(pos, ast, expected)) + return false; + + current_custom_name = ast->as().value.get(); + } + + res->server_type = ServerType(current_type, current_custom_name); + + break; + } + default: { if (!parseQueryWithOnCluster(res, pos, expected)) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 82ca7bc0688..e411574bd65 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -3,8 +3,8 @@ set(SRCS) clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) -clickhouse_add_executable(select_parser select_parser.cpp ${SRCS}) +clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") target_link_libraries(select_parser PRIVATE clickhouse_parsers) -clickhouse_add_executable(create_parser create_parser.cpp ${SRCS}) +clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") target_link_libraries(create_parser PRIVATE clickhouse_parsers) diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp new file mode 100644 index 00000000000..c6916ee39d9 --- /dev/null +++ b/src/Server/ServerType.cpp @@ -0,0 +1,138 @@ +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + std::vector getTypeIndexToTypeName() + { + constexpr std::size_t types_size = magic_enum::enum_count(); + + std::vector type_index_to_type_name; + type_index_to_type_name.resize(types_size); + + auto entries = magic_enum::enum_entries(); + for (const auto & [entry, str] : entries) + { + auto str_copy = String(str); + std::replace(str_copy.begin(), str_copy.end(), '_', ' '); + type_index_to_type_name[static_cast(entry)] = std::move(str_copy); + } + + return type_index_to_type_name; + } +} + +const char * ServerType::serverTypeToString(ServerType::Type type) +{ + /** During parsing if SystemQuery is not parsed properly it is added to Expected variants as description check IParser.h. + * Description string must be statically allocated. + */ + static std::vector type_index_to_type_name = getTypeIndexToTypeName(); + const auto & type_name = type_index_to_type_name[static_cast(type)]; + return type_name.data(); +} + +bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) const +{ + if (type == Type::QUERIES_ALL) + return true; + + if (type == Type::QUERIES_DEFAULT) + { + switch (server_type) + { + case Type::TCP: + case Type::TCP_WITH_PROXY: + case Type::TCP_SECURE: + case Type::HTTP: + case Type::HTTPS: + case Type::MYSQL: + case Type::GRPC: + case Type::POSTGRESQL: + case Type::PROMETHEUS: + case Type::INTERSERVER_HTTP: + case Type::INTERSERVER_HTTPS: + return true; + default: + return false; + } + } + + if (type == Type::QUERIES_CUSTOM) + { + switch (server_type) + { + case Type::CUSTOM: + return true; + default: + return false; + } + } + + return type == server_type && custom_name == custom_name_; +} + +bool ServerType::shouldStop(const std::string & port_name) const +{ + Type port_type; + std::string port_custom_name; + + if (port_name == "http_port") + port_type = Type::HTTP; + + else if (port_name == "https_port") + port_type = Type::HTTPS; + + else if (port_name == "tcp_port") + port_type = Type::TCP; + + else if (port_name == "tcp_with_proxy_port") + port_type = Type::TCP_WITH_PROXY; + + else if (port_name == "tcp_port_secure") + port_type = Type::TCP_SECURE; + + else if (port_name == "mysql_port") + port_type = Type::MYSQL; + + else if (port_name == "postgresql_port") + port_type = Type::POSTGRESQL; + + else if (port_name == "grpc_port") + port_type = Type::GRPC; + + else if (port_name == "prometheus.port") + port_type = Type::PROMETHEUS; + + else if (port_name == "interserver_http_port") + port_type = Type::INTERSERVER_HTTP; + + else if (port_name == "interserver_https_port") + port_type = Type::INTERSERVER_HTTPS; + + else if (port_name.starts_with("protocols.") && port_name.ends_with(".port")) + { + constexpr size_t protocols_size = std::string_view("protocols.").size(); + constexpr size_t port_size = std::string_view("protocols.").size(); + + port_type = Type::CUSTOM; + port_custom_name = port_name.substr(protocols_size, port_name.size() - port_size); + } + else + port_type = Type::UNKNOWN; + + if (port_type == Type::UNKNOWN) + return false; + + return shouldStart(type, port_custom_name); +} + +} diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h new file mode 100644 index 00000000000..345d1a10119 --- /dev/null +++ b/src/Server/ServerType.h @@ -0,0 +1,44 @@ +#pragma once + +#include +namespace DB +{ + +class ServerType +{ +public: + + enum Type + { + UNKNOWN, + TCP, + TCP_WITH_PROXY, + TCP_SECURE, + HTTP, + HTTPS, + MYSQL, + GRPC, + POSTGRESQL, + PROMETHEUS, + CUSTOM, + INTERSERVER_HTTP, + INTERSERVER_HTTPS, + QUERIES_ALL, + QUERIES_DEFAULT, + QUERIES_CUSTOM, + END + }; + + ServerType() = default; + explicit ServerType(Type type_, const std::string & custom_name_ = "") : type(type_), custom_name(custom_name_) {} + + static const char * serverTypeToString(Type type); + + bool shouldStart(Type server_type, const std::string & custom_name_ = "") const; + bool shouldStop(const std::string & port_name) const; + + Type type; + std::string custom_name; +}; + +} diff --git a/tests/integration/test_system_start_stop_listen/__init__.py b/tests/integration/test_system_start_stop_listen/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_start_stop_listen/configs/cluster.xml b/tests/integration/test_system_start_stop_listen/configs/cluster.xml new file mode 100644 index 00000000000..93d8f890f40 --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/configs/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py new file mode 100644 index 00000000000..ec1a000c599 --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -0,0 +1,40 @@ +#!/usr/bin/env python3 + + +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +import random +import string +import json + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/cluster.xml"], with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/cluster.xml"], with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_system_start_stop_listen_queries(started_cluster): + node1.query("SYSTEM STOP LISTEN QUERIES ALL") + + assert "Connection refused" in node1.query_and_get_error("SELECT 1", timeout=3) + + node2.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + node1.query("SELECT 1") diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index e6f7fa1ed2b..db0f2d8235b 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -138,6 +138,7 @@ 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 SYSTEM FAILPOINT ['SYSTEM ENABLE FAILPOINT','SYSTEM DISABLE FAILPOINT'] GLOBAL SYSTEM +SYSTEM LISTEN ['SYSTEM START LISTEN','SYSTEM STOP LISTEN'] GLOBAL SYSTEM SYSTEM [] \N ALL dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL displaySecretsInShowAndSelect [] GLOBAL ALL 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 d58c76260c5..46d1f0e3a0b 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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166), + `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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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 LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -584,10 +584,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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166), + `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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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 LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), `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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166)) + `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' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 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 LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 5aeeaebd00c90385b1d9d8e3c6b13d298240c752 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 26 Jul 2023 17:09:23 -0300 Subject: [PATCH 779/871] init and destroy ares channel on demand.. --- src/Common/CaresPTRResolver.cpp | 71 ++++++++++--------- src/Common/CaresPTRResolver.h | 29 ++++---- .../tests/gtest_dns_reverse_resolve.cpp | 35 ++++----- 3 files changed, 71 insertions(+), 64 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index fee4d01eb37..bf61e940745 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -41,9 +41,25 @@ namespace DB } } - std::mutex CaresPTRResolver::mutex; + struct AresChannelRAII + { + AresChannelRAII() + { + if (ares_init(&channel) != ARES_SUCCESS) + { + throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares channel"); + } + } - CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) + ~AresChannelRAII() + { + ares_destroy(channel); + } + + ares_channel channel; + }; + + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) { /* * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. @@ -57,34 +73,22 @@ namespace DB * */ static const auto library_init_result = ares_library_init(ARES_LIB_INIT_ALL); - if (library_init_result != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) + if (library_init_result != ARES_SUCCESS) { throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares"); } } - CaresPTRResolver::~CaresPTRResolver() - { - ares_destroy(channel); - /* - * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver - * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. - * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen - * until the end of the program. Hence, ares_library_cleanup() will not be called. - * */ - } - std::unordered_set CaresPTRResolver::resolve(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve(ip, ptr_records); + resolve(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } @@ -93,22 +97,21 @@ namespace DB std::unordered_set CaresPTRResolver::resolve_v6(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve_v6(ip, ptr_records); + resolve_v6(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } return ptr_records; } - void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response, ares_channel channel) { in_addr addr; @@ -117,7 +120,7 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); } - void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel) { in6_addr addr; inet_pton(AF_INET6, ip.c_str(), &addr); @@ -125,15 +128,15 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); } - bool CaresPTRResolver::wait_and_process() + bool CaresPTRResolver::wait_and_process(ares_channel channel) { int sockets[ARES_GETSOCK_MAXNUM]; pollfd pollfd[ARES_GETSOCK_MAXNUM]; while (true) { - auto readable_sockets = get_readable_sockets(sockets, pollfd); - auto timeout = calculate_timeout(); + auto readable_sockets = get_readable_sockets(sockets, pollfd, channel); + auto timeout = calculate_timeout(channel); int number_of_fds_ready = 0; if (!readable_sockets.empty()) @@ -158,11 +161,11 @@ namespace DB if (number_of_fds_ready > 0) { - process_readable_sockets(readable_sockets); + process_readable_sockets(readable_sockets, channel); } else { - process_possible_timeout(); + process_possible_timeout(channel); break; } } @@ -170,12 +173,12 @@ namespace DB return true; } - void CaresPTRResolver::cancel_requests() + void CaresPTRResolver::cancel_requests(ares_channel channel) { ares_cancel(channel); } - std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd) + std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel) { int sockets_bitmask = ares_getsock(channel, sockets, ARES_GETSOCK_MAXNUM); @@ -205,7 +208,7 @@ namespace DB return std::span(pollfd, number_of_sockets_to_poll); } - int64_t CaresPTRResolver::calculate_timeout() + int64_t CaresPTRResolver::calculate_timeout(ares_channel channel) { timeval tv; if (auto * tvp = ares_timeout(channel, nullptr, &tv)) @@ -218,14 +221,14 @@ namespace DB return 0; } - void CaresPTRResolver::process_possible_timeout() + void CaresPTRResolver::process_possible_timeout(ares_channel channel) { /* Call ares_process() unconditonally here, even if we simply timed out above, as otherwise the ares name resolve won't timeout! */ ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD); } - void CaresPTRResolver::process_readable_sockets(std::span readable_sockets) + void CaresPTRResolver::process_readable_sockets(std::span readable_sockets, ares_channel channel) { for (auto readable_socket : readable_sockets) { diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h index 454509ae43c..24a5e422ca8 100644 --- a/src/Common/CaresPTRResolver.h +++ b/src/Common/CaresPTRResolver.h @@ -28,32 +28,35 @@ namespace DB public: explicit CaresPTRResolver(provider_token); - ~CaresPTRResolver() override; + + /* + * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver + * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. + * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen + * until the end of the program. Hence, ares_library_cleanup() will not be called. + * */ + ~CaresPTRResolver() override = default; std::unordered_set resolve(const std::string & ip) override; std::unordered_set resolve_v6(const std::string & ip) override; private: - bool wait_and_process(); + bool wait_and_process(ares_channel channel); - void cancel_requests(); + void cancel_requests(ares_channel channel); - void resolve(const std::string & ip, std::unordered_set & response); + void resolve(const std::string & ip, std::unordered_set & response, ares_channel channel); - void resolve_v6(const std::string & ip, std::unordered_set & response); + void resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel); - std::span get_readable_sockets(int * sockets, pollfd * pollfd); + std::span get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel); - int64_t calculate_timeout(); + int64_t calculate_timeout(ares_channel channel); - void process_possible_timeout(); + void process_possible_timeout(ares_channel channel); - void process_readable_sockets(std::span readable_sockets); - - ares_channel channel; - - static std::mutex mutex; + void process_readable_sockets(std::span readable_sockets, ares_channel channel); }; } diff --git a/src/Common/tests/gtest_dns_reverse_resolve.cpp b/src/Common/tests/gtest_dns_reverse_resolve.cpp index 08351564eaf..de33deddac3 100644 --- a/src/Common/tests/gtest_dns_reverse_resolve.cpp +++ b/src/Common/tests/gtest_dns_reverse_resolve.cpp @@ -9,34 +9,35 @@ namespace DB { TEST(Common, ReverseDNS) { - auto addresses = std::vector({ - "8.8.8.8", "2001:4860:4860::8888", // dns.google - "142.250.219.35", // google.com - "157.240.12.35", // facebook - "208.84.244.116", "2600:1419:c400::214:c410", //www.terra.com.br, - "127.0.0.1", "::1" - }); - auto func = [&]() { // Good random seed, good engine auto rnd1 = std::mt19937(std::random_device{}()); - for (int i = 0; i < 50; ++i) + for (int i = 0; i < 10; ++i) { auto & dns_resolver_instance = DNSResolver::instance(); -// unfortunately, DNS cache can't be disabled because we might end up causing a DDoS attack -// dns_resolver_instance.setDisableCacheFlag(); + dns_resolver_instance.setDisableCacheFlag(); - auto addr_index = rnd1() % addresses.size(); + auto val1 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val2 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val3 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val4 = rnd1() % static_cast((pow(2, 31) - 1)); - [[maybe_unused]] auto result = dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ addresses[addr_index] }); + uint32_t ipv4_buffer[1] = { + static_cast(val1) + }; -// will not assert either because some of the IP addresses might change in the future and -// this test will become flaky -// ASSERT_TRUE(!result.empty()); + uint32_t ipv6_buffer[4] = { + static_cast(val1), + static_cast(val2), + static_cast(val3), + static_cast(val4) + }; + + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv4_buffer, sizeof(ipv4_buffer)}); + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv6_buffer, sizeof(ipv6_buffer)}); } - }; auto number_of_threads = 200u; From 954a1d3edec8117a135c23b7ec60065249fa0f02 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 23:38:14 +0300 Subject: [PATCH 780/871] Improve logging macros (#52519) * improve logging macros * fix * Update logger_useful.h * fix * fix --- base/poco/Foundation/include/Poco/Message.h | 2 + base/poco/Foundation/src/Message.cpp | 13 +++ src/Common/LoggingFormatStringHelpers.h | 63 ++++++++---- src/Common/logger_useful.h | 97 ++++++++++++++----- src/Common/tests/gtest_log.cpp | 53 ++++++++++ .../01164_detach_attach_partition_race.sh | 2 +- 6 files changed, 189 insertions(+), 41 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Message.h b/base/poco/Foundation/include/Poco/Message.h index e8f04888ab4..282c7fb5fd1 100644 --- a/base/poco/Foundation/include/Poco/Message.h +++ b/base/poco/Foundation/include/Poco/Message.h @@ -67,6 +67,8 @@ public: Message( const std::string & source, const std::string & text, Priority prio, const char * file, int line, std::string_view fmt_str = {}); + Message( + std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str); /// Creates a Message with the given source, text, priority, /// source file path and line. /// diff --git a/base/poco/Foundation/src/Message.cpp b/base/poco/Foundation/src/Message.cpp index 663c96e47a2..54118cc0fc5 100644 --- a/base/poco/Foundation/src/Message.cpp +++ b/base/poco/Foundation/src/Message.cpp @@ -60,6 +60,19 @@ Message::Message(const std::string& source, const std::string& text, Priority pr } +Message::Message(std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str): + _source(std::move(source)), + _text(std::move(text)), + _prio(prio), + _tid(0), + _file(file), + _line(line), + _pMap(0), + _fmt_str(fmt_str) +{ + init(); +} + Message::Message(const Message& msg): _source(msg._source), _text(msg._text), diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 3afa3fb089d..6dffd202807 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -43,6 +43,17 @@ struct PreformattedMessage operator const std::string & () const { return text; } operator std::string () && { return std::move(text); } operator fmt::format_string<> () const { UNREACHABLE(); } + + void apply(std::string & out_text, std::string_view & out_format_string) const & + { + out_text = text; + out_format_string = format_string; + } + void apply(std::string & out_text, std::string_view & out_format_string) && + { + out_text = std::move(text); + out_format_string = format_string; + } }; template @@ -99,10 +110,33 @@ template constexpr std::string_view tryGetStaticFormatString(T && x } } +/// Constexpr ifs are not like ifdefs, and compiler still checks that unneeded code can be compiled +/// This template is useful to avoid compilation failures when condition of some "constexpr if" is false +template struct ConstexprIfsAreNotIfdefs +{ + template constexpr static std::string_view getStaticFormatString(T &&) { return {}; } + template static PreformattedMessage getPreformatted(T &&) { return {}; } +}; + +template<> struct ConstexprIfsAreNotIfdefs +{ + template consteval static std::string_view getStaticFormatString(T && x) + { + /// See tryGetStaticFormatString(...) + static_assert(!std::is_same_v>); + static_assert(std::is_nothrow_convertible::value); + static_assert(!std::is_pointer::value); + return std::string_view(x); + } + + template static T && getPreformatted(T && x) { return std::forward(x); } +}; + template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } /// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor template constexpr auto firstArg(fmt::basic_runtime && data, Ts &&...) { return data.str.data(); } +template constexpr auto firstArg(const fmt::basic_runtime & data, Ts &&...) { return data.str.data(); } consteval ssize_t formatStringCountArgsNum(const char * const str, size_t len) { @@ -142,26 +176,19 @@ consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs) functionThatFailsCompilationOfConstevalFunctions("unexpected number of arguments in a format string"); } -template -struct CheckArgsNumHelperImpl +template +consteval void formatStringCheckArgsNum(T && str, size_t nargs) { - template - consteval CheckArgsNumHelperImpl(T && str) - { - formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), sizeof...(Args)); - } - - /// No checks for fmt::runtime and PreformattedMessage - template CheckArgsNumHelperImpl(fmt::basic_runtime &&) {} - template<> CheckArgsNumHelperImpl(PreformattedMessage &) {} - template<> CheckArgsNumHelperImpl(const PreformattedMessage &) {} - template<> CheckArgsNumHelperImpl(PreformattedMessage &&) {} - -}; - -template using CheckArgsNumHelper = CheckArgsNumHelperImpl...>; -template void formatStringCheckArgsNum(CheckArgsNumHelper, Args &&...) {} + formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), nargs); +} +template inline void formatStringCheckArgsNum(fmt::basic_runtime &&, size_t) {} +template<> inline void formatStringCheckArgsNum(PreformattedMessage &, size_t) {} +template<> inline void formatStringCheckArgsNum(const PreformattedMessage &, size_t) {} +template<> inline void formatStringCheckArgsNum(PreformattedMessage &&, size_t) {} +template struct FormatStringTypeInfo{ static constexpr bool is_static = true; static constexpr bool has_format = true; }; +template struct FormatStringTypeInfo> { static constexpr bool is_static = false; static constexpr bool has_format = false; }; +template<> struct FormatStringTypeInfo { static constexpr bool is_static = false; static constexpr bool has_format = true; }; /// This wrapper helps to avoid too frequent and noisy log messages. /// For each pair (logger_name, format_string) it remembers when such a message was logged the last time. diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 3ebb1d25075..d9fe5ac9190 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -1,7 +1,7 @@ #pragma once /// Macros for convenient usage of Poco logger. - +#include #include #include #include @@ -28,33 +28,86 @@ namespace #define LOG_IMPL_FIRST_ARG(X, ...) X +/// Copy-paste from contrib/libpq/include/c.h +/// There's no easy way to count the number of arguments without evaluating these arguments... +#define CH_VA_ARGS_NARGS(...) \ + CH_VA_ARGS_NARGS_(__VA_ARGS__, \ + 63,62,61,60, \ + 59,58,57,56,55,54,53,52,51,50, \ + 49,48,47,46,45,44,43,42,41,40, \ + 39,38,37,36,35,34,33,32,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, 1, 0) +#define CH_VA_ARGS_NARGS_( \ + _01,_02,_03,_04,_05,_06,_07,_08,_09,_10, \ + _11,_12,_13,_14,_15,_16,_17,_18,_19,_20, \ + _21,_22,_23,_24,_25,_26,_27,_28,_29,_30, \ + _31,_32,_33,_34,_35,_36,_37,_38,_39,_40, \ + _41,_42,_43,_44,_45,_46,_47,_48,_49,_50, \ + _51,_52,_53,_54,_55,_56,_57,_58,_59,_60, \ + _61,_62,_63, N, ...) \ + (N) + +#define LINE_NUM_AS_STRING_IMPL2(x) #x +#define LINE_NUM_AS_STRING_IMPL(x) LINE_NUM_AS_STRING_IMPL2(x) +#define LINE_NUM_AS_STRING LINE_NUM_AS_STRING_IMPL(__LINE__) +#define MESSAGE_FOR_EXCEPTION_ON_LOGGING "Failed to write a log message: " __FILE__ ":" LINE_NUM_AS_STRING "\n" + /// Logs a message to a specified logger with that level. /// If more than one argument is provided, /// the first argument is interpreted as a template with {}-substitutions /// and the latter arguments are treated as values to substitute. /// If only one argument is provided, it is treated as a message without substitutions. -#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ -{ \ - auto _logger = ::getLogger(logger); \ - const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ - (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ - if (_is_clients_log || _logger->is((PRIORITY))) \ - { \ - std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ - formatStringCheckArgsNum(__VA_ARGS__); \ - if (auto _channel = _logger->getChannel()) \ - { \ - std::string file_function; \ - file_function += __FILE__; \ - file_function += "; "; \ - file_function += __PRETTY_FUNCTION__; \ - Poco::Message poco_message(_logger->name(), formatted_message, \ - (PRIORITY), file_function.c_str(), __LINE__, tryGetStaticFormatString(LOG_IMPL_FIRST_ARG(__VA_ARGS__))); \ - _channel->log(poco_message); \ - } \ - ProfileEvents::incrementForLogMessage(PRIORITY); \ - } \ +#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ +{ \ + auto _logger = ::getLogger(logger); \ + const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ + (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ + if (!_is_clients_log && !_logger->is((PRIORITY))) \ + break; \ + \ + try \ + { \ + ProfileEvents::incrementForLogMessage(PRIORITY); \ + auto _channel = _logger->getChannel(); \ + if (!_channel) \ + break; \ + \ + constexpr size_t _nargs = CH_VA_ARGS_NARGS(__VA_ARGS__); \ + using LogTypeInfo = FormatStringTypeInfo>; \ + \ + std::string_view _format_string; \ + std::string _formatted_message; \ + \ + if constexpr (LogTypeInfo::is_static) \ + { \ + formatStringCheckArgsNum(LOG_IMPL_FIRST_ARG(__VA_ARGS__), _nargs - 1); \ + _format_string = ConstexprIfsAreNotIfdefs::getStaticFormatString(LOG_IMPL_FIRST_ARG(__VA_ARGS__)); \ + } \ + \ + constexpr bool is_preformatted_message = !LogTypeInfo::is_static && LogTypeInfo::has_format; \ + if constexpr (is_preformatted_message) \ + { \ + static_assert(_nargs == 1 || !is_preformatted_message); \ + ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string); \ + } \ + else \ + { \ + _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : fmt::format(__VA_ARGS__); \ + } \ + \ + std::string _file_function = __FILE__ "; "; \ + _file_function += __PRETTY_FUNCTION__; \ + Poco::Message _poco_message(_logger->name(), std::move(_formatted_message), \ + (PRIORITY), _file_function.c_str(), __LINE__, _format_string); \ + _channel->log(_poco_message); \ + } \ + catch (...) \ + { \ + ::write(STDERR_FILENO, static_cast(MESSAGE_FOR_EXCEPTION_ON_LOGGING), sizeof(MESSAGE_FOR_EXCEPTION_ON_LOGGING)); \ + } \ } while (false) diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index f92866626f9..e755c22ba75 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -50,3 +51,55 @@ TEST(Logger, TestLog) } } + +static size_t global_counter = 0; + +static std::string getLogMessage() +{ + ++global_counter; + return "test1 " + std::to_string(thread_local_rng()); +} + +static size_t getLogMessageParam() +{ + ++global_counter; + return thread_local_rng(); +} + +static PreformattedMessage getPreformatted() +{ + ++global_counter; + return PreformattedMessage::create("test3 {}", thread_local_rng()); +} + +static size_t getLogMessageParamOrThrow() +{ + size_t x = thread_local_rng(); + if (x % 1000 == 0) + return x; + throw Poco::Exception("error", 42); +} + +TEST(Logger, SideEffects) +{ + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); + auto * log = &Poco::Logger::create("Logger", my_channel.get()); + log->setLevel("trace"); + + /// Ensure that parameters are evaluated only once + global_counter = 0; + LOG_TRACE(log, fmt::runtime(getLogMessage())); + EXPECT_EQ(global_counter, 1); + LOG_TRACE(log, "test2 {}", getLogMessageParam()); + EXPECT_EQ(global_counter, 2); + LOG_TRACE(log, getPreformatted()); + EXPECT_EQ(global_counter, 3); + + auto var = PreformattedMessage::create("test4 {}", thread_local_rng()); + LOG_TRACE(log, var); + EXPECT_EQ(var.text.starts_with("test4 "), true); + EXPECT_EQ(var.format_string, "test4 {}"); + + LOG_TRACE(log, "test no throw {}", getLogMessageParamOrThrow()); +} diff --git a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh index e645cb5aae7..07b39723c37 100755 --- a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh +++ b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists mt" -$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n settings parts_to_throw_insert=1000" +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n settings parts_to_throw_insert=5000" $CLICKHOUSE_CLIENT -q "insert into mt values (1)" $CLICKHOUSE_CLIENT -q "insert into mt values (2)" $CLICKHOUSE_CLIENT -q "insert into mt values (3)" From 6aab4cc83508093d19ed84130be483e73c08b324 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 05:25:40 +0200 Subject: [PATCH 781/871] Check for unexpected cyrillic --- src/Compression/CompressionCodecEncrypted.cpp | 2 +- src/Disks/DiskEncrypted.cpp | 4 ++-- src/IO/examples/read_buffer.cpp | 4 ++-- src/IO/examples/write_buffer.cpp | 2 +- src/IO/examples/write_buffer_perf.cpp | 2 +- src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp | 2 +- utils/check-style/check-style | 3 +++ 7 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 3f4e35a78a4..34b621291fd 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -493,7 +493,7 @@ void CompressionCodecEncrypted::Configuration::loadImpl( /// If there is only one key with non zero ID, curren_key_id should be defined. if (new_params->keys_storage[method].size() == 1 && !new_params->keys_storage[method].contains(0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. current_key_id is required"); } /// Try to find which key will be used for encryption. If there is no current_key and only one key without id diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 441e639b967..aa9c4a92adc 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -433,10 +433,10 @@ void DiskEncrypted::applyNewSettings( { auto new_settings = parseDiskEncryptedSettings(name, config, config_prefix, disk_map); if (new_settings->wrapped_disk != delegate) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging wrapped disk on the fly is not supported. Disk {}", name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing wrapped disk on the fly is not supported. Disk {}", name); if (new_settings->disk_path != disk_path) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging disk path on the fly is not supported. Disk {}", name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing disk path on the fly is not supported. Disk {}", name); current_settings.set(std::move(new_settings)); IDisk::applyNewSettings(config, context, config_prefix, disk_map); diff --git a/src/IO/examples/read_buffer.cpp b/src/IO/examples/read_buffer.cpp index 85675c0d613..221da24715b 100644 --- a/src/IO/examples/read_buffer.cpp +++ b/src/IO/examples/read_buffer.cpp @@ -40,7 +40,7 @@ int readAndPrint(DB::ReadBuffer & in) int main(int, char **) { { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'"; DB::ReadBufferFromString in(s); if (readAndPrint(in)) std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl; @@ -49,7 +49,7 @@ int main(int, char **) std::shared_ptr in; { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'"; in = std::make_shared(s); } if (readAndPrint(*in)) diff --git a/src/IO/examples/write_buffer.cpp b/src/IO/examples/write_buffer.cpp index bca0be24b1a..999f9b1bb34 100644 --- a/src/IO/examples/write_buffer.cpp +++ b/src/IO/examples/write_buffer.cpp @@ -14,7 +14,7 @@ int main(int, char **) { DB::Int64 a = -123456; DB::Float64 b = 123.456; - DB::String c = "вася пе\tтя"; + DB::String c = "вася pe\ttya"; DB::String d = "'xyz\\"; std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM diff --git a/src/IO/examples/write_buffer_perf.cpp b/src/IO/examples/write_buffer_perf.cpp index 0b3d0a61241..3f57ddb9a4f 100644 --- a/src/IO/examples/write_buffer_perf.cpp +++ b/src/IO/examples/write_buffer_perf.cpp @@ -14,7 +14,7 @@ int main(int, char **) { DB::Int64 a = -123456; DB::Float64 b = 123.456; - DB::String c = "вася пе\tтя"; + DB::String c = "вася pe\ttya"; DB::String d = "'xyz\\"; std::ofstream s("test"); diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 229a0630328..cff83b0ad3b 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -180,7 +180,7 @@ Chunk ParquetMetadataInputFormat::generate() else if (name == names[3]) { auto column = types[3]->createColumn(); - /// Version сan be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6). + /// Version can be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6). String version = metadata->version() == parquet::ParquetVersion::PARQUET_1_0 ? "1.0" : "2.6"; assert_cast(*column).insertData(version.data(), version.size()); res.addColumn(std::move(column)); diff --git a/utils/check-style/check-style b/utils/check-style/check-style index c28ca1cfc8a..67c185a0b54 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -413,3 +413,6 @@ find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep - # 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!'" + +# Cyrillic characters hiding inside Latin. +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place." From d35c87c1bdf4cba8848e94acd0ed8ec1f6b34502 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 05:06:16 +0000 Subject: [PATCH 782/871] allow positional options for clickhouse-local and populate global udf settings --- programs/local/LocalServer.cpp | 15 +++++++++++++++ src/Client/ClientBase.cpp | 5 ----- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..3ee268aed0e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -266,6 +266,10 @@ void LocalServer::tryInitPath() global_context->setUserFilesPath(""); // user's files are everywhere + std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/"); + global_context->setUserScriptsPath(user_scripts_path); + fs::create_directories(user_scripts_path); + /// top_level_domains_lists const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); if (!top_level_domains_path.empty()) @@ -490,6 +494,17 @@ try applyCmdSettings(global_context); + /// try to load user defined executable functions, throw on error and die + try + { + global_context->loadOrReloadUserDefinedExecutableFunctions(config()); + } + catch (...) + { + tryLogCurrentException(&logger(), "Caught exception while loading user defined executable functions."); + throw; + } + if (is_interactive) { clearTerminal(); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 3e964d5c6a3..496fc8fce0a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,11 +2624,6 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } - /// Check positional options (options after ' -- ', ex: clickhouse-client -- ). - unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - if (unrecognized_options.size() > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); - po::store(parsed, options); } From 65ffe91bf26a3429fe691c755736867e7819d2f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 07:13:26 +0200 Subject: [PATCH 783/871] Fix double whitespace --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index b2c75db0e54..c661e6b782d 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1011,7 +1011,7 @@ private: using ConfluentSchemaRegistry = AvroConfluentRowInputFormat::SchemaRegistry; #define SCHEMA_REGISTRY_CACHE_MAX_SIZE 1000 /// Cache of Schema Registry URL -> SchemaRegistry -static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); +static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); static std::shared_ptr getConfluentSchemaRegistry(const FormatSettings & format_settings) { From d2d7139da3af470a49267047f0ea45f652d59e45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 08:58:23 +0200 Subject: [PATCH 784/871] Changelog for 23.7 --- CHANGELOG.md | 175 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 175 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf6b309ef2c..878edfa4add 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.7, 2023-07-27](#237)**
**[ClickHouse release v23.6, 2023-06-30](#236)**
**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
@@ -9,6 +10,180 @@ # 2023 Changelog +### ClickHouse release 23.7, 2023-07-27 + +#### Backward Incompatible Change +* Add `NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after *downgrade* (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch Avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Add a column `is_obsolete` to `system.settings` and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file. Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from `` section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Grace Hash Join algorithm is now applicable to FULL and RIGHT JOINs. [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add `SYSTEM STOP LISTEN` query for more graceful termination. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `input_format_csv_allow_variable_number_of_columns` options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function `substring_index`, as in Spark or MySQL. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* A system table `jemalloc_bins` to show stats for jemalloc bins. Example `SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10`. Enjoy. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add `RowBinaryWithDefaults` format with extra byte before each column as a flag for using the column's default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new `initcap` / `initcapUTF8` functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage statistic to HTTP headers. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new `hasSubsequence` (+`CaseInsensitive` and `UTF8` versions) functions to match subsequences in strings. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Implement KQL-style formatting for the `Interval` data type. This is only needed for compatibility with the `Kusto` query language. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Aliases `current_database` and a new function `current_schemas` for compatibility with PostgreSQL. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Support `async_deduplication_token` for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Performance Improvement +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Move conditions with primary key columns to the end of PREWHERE chain. The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* Speed up `COUNT(DISTINCT)` for String types by inlining SipHash. The performance experiments of *OnTime* on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of *11.6%* to the QPS of the query *Q8* while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1`. This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823). This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173). [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in `FileCache::loadMetadata` - this speeds up server startup if the filesystem cache is configured. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Improvement +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow filtering HTTP headers for the URL/S3 table functions with the new `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues in `geoHash` functions. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Log async insert flush queries into `system.query_log`. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Functions `date_diff` and `age` now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: a Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column `modification_time` into `system.parts_columns`. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function `range` of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of the statement `SHOW INDEX`. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Cache dictionary now requests only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed the case when settings were not applied for EXPLAIN query when FORMAT was provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-Int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Log messages are written to the `system.text_log` from the server startup. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception was thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Avro input format now supports Union even if it contains only a single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093). [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). + +#### Experimental Feature +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. +* (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now unquoted UTF-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* The `session_timezone` setting (new in version 23.6) is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add integration test check with the enabled Analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Update Cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix MaterializedPostgreSQL syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release 23.6, 2023-06-29 #### Backward Incompatible Change From ac51ade45c5581d031c382277b550b2fb2f873fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 27 Jul 2023 08:45:53 +0000 Subject: [PATCH 785/871] Do not run the test without Rust libraries --- tests/queries/0_stateless/02833_local_with_dialect.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh index 2a2e1b09459..012a6d91269 100755 --- a/tests/queries/0_stateless/02833_local_with_dialect.sh +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f61f36800ccd028bb9cf4ef402275006faf2facb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Jul 2023 08:48:23 +0000 Subject: [PATCH 786/871] Fix style --- .../Formats/Impl/Parquet/PrepareForWrite.cpp | 14 +++++++------- src/Processors/Formats/Impl/Parquet/Write.cpp | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index 0700fc8491c..bc4c9ca3b72 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -303,14 +303,14 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin } else { - types(T::INT32, C::UINT_8 , int_type(8 , false)); + types(T::INT32, C::UINT_8, int_type(8, false)); } break; case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; - case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; - case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break; + case TypeIndex::Int8: types(T::INT32, C::INT_8, int_type(8, true)); break; + case TypeIndex::Int16: types(T::INT32, C::INT_16, int_type(16, true)); break; case TypeIndex::Int32: types(T::INT32); break; case TypeIndex::Int64: types(T::INT64); break; case TypeIndex::Float32: types(T::FLOAT); break; @@ -319,8 +319,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin /// These don't have suitable parquet logical types, so we write them as plain numbers. /// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum /// values in advance as part of the data type.) - case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8 - case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16 + case TypeIndex::Enum8: types(T::INT32, C::INT_8, int_type(8, true)); break; // Int8 + case TypeIndex::Enum16: types(T::INT32, C::INT_16, int_type(16, true)); break; // Int16 case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16 case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 @@ -392,8 +392,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin case TypeIndex::Int256: fixed_string(32); break; case TypeIndex::IPv6: fixed_string(16); break; - case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break; - case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal32: decimal(4, getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal64: decimal(8, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 47ef0c53ab5..5ebf2be76d2 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -755,20 +755,20 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option writeColumnImpl(s, options, out, ConverterNumeric, bool, bool>(s.primitive_column)); else - N(UInt8 , Int32Type); + N(UInt8, Int32Type); break; case TypeIndex::UInt16 : N(UInt16, Int32Type); break; case TypeIndex::UInt32 : N(UInt32, Int32Type); break; case TypeIndex::UInt64 : N(UInt64, Int64Type); break; - case TypeIndex::Int8 : N(Int8 , Int32Type); break; - case TypeIndex::Int16 : N(Int16 , Int32Type); break; - case TypeIndex::Int32 : N(Int32 , Int32Type); break; - case TypeIndex::Int64 : N(Int64 , Int64Type); break; + case TypeIndex::Int8 : N(Int8, Int32Type); break; + case TypeIndex::Int16 : N(Int16, Int32Type); break; + case TypeIndex::Int32 : N(Int32, Int32Type); break; + case TypeIndex::Int64 : N(Int64, Int64Type); break; - case TypeIndex::Enum8: N(Int8 , Int32Type); break; - case TypeIndex::Enum16: N(Int16 , Int32Type); break; + case TypeIndex::Enum8: N(Int8, Int32Type); break; + case TypeIndex::Enum16: N(Int16, Int32Type); break; case TypeIndex::Date: N(UInt16, Int32Type); break; - case TypeIndex::Date32: N(Int32 , Int32Type); break; + case TypeIndex::Date32: N(Int32, Int32Type); break; case TypeIndex::DateTime: N(UInt32, Int32Type); break; #undef N From 2b18872e86898fe0c0ee40ddecf05c29088a7aca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 08:55:19 +0000 Subject: [PATCH 787/871] Incorporate review feedback --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 8 ++++---- src/Interpreters/QueryLog.cpp | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index b8dc0c0224c..835c79129de 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'None' = 1` = The query result was neither written into nor read from the query cache. - - `'Write' = 1` = The query result was written into the query cache. - - `'Read' = 1` = The query result was read from the query cache. - - `'Unknown' = 1` = Unknown status. + - `'Unknown' = 1` = Status unknown. + - `'None' = 2` = The query result was neither written into nor read from the query cache. + - `'Write' = 3` = The query result was written into the query cache. + - `'Read' = 4` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 973015b8003..5fe756268f2 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -27,10 +27,10 @@ public: enum class Usage { /// starts at 1 for compatibitity with DataTypeEnum8 - None = 1, /// query result neither written nor read into/from query cache - Write, /// query result wrote into query cache - Read, /// query result read from query cache - Unknown, /// we don't know what what happened + Unknown = 1, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index c3294512f14..df21e82305a 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -44,10 +44,10 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() auto query_cache_usage_datatype = std::make_shared( DataTypeEnum8::Values { + {"Unknown", static_cast(QueryCache::Usage::Unknown)}, {"None", static_cast(QueryCache::Usage::None)}, {"Write", static_cast(QueryCache::Usage::Write)}, - {"Read", static_cast(QueryCache::Usage::Read)}, - {"Unknown", static_cast(QueryCache::Usage::Unknown)} + {"Read", static_cast(QueryCache::Usage::Read)} }); auto low_cardinality_string = std::make_shared(std::make_shared()); From 043ad45ec3f4f2d1ff8b619da4c6ec5cb2b8f0d0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 09:18:24 +0000 Subject: [PATCH 788/871] Incorporate review feedback, pt. II --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 835c79129de..c6f565b8748 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'Unknown' = 1` = Status unknown. - - `'None' = 2` = The query result was neither written into nor read from the query cache. - - `'Write' = 3` = The query result was written into the query cache. - - `'Read' = 4` = The query result was read from the query cache. + - `'Unknown'` = Status unknown. + - `'None'` = The query result was neither written into nor read from the query cache. + - `'Write'` = The query result was written into the query cache. + - `'Read'` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 5fe756268f2..c2de8ca22dd 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -26,11 +26,10 @@ class QueryCache public: enum class Usage { - /// starts at 1 for compatibitity with DataTypeEnum8 - Unknown = 1, /// we don't know what what happened - None, /// query result neither written nor read into/from query cache - Write, /// query result written into query cache - Read, /// query result read from query cache + Unknown, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. From 578ea9c5e8a0e9eed4da2afa15c84951aba9d688 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 27 Jul 2023 09:50:42 +0000 Subject: [PATCH 789/871] Update documentation --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 62f931a76b4..63bf3cfeb5c 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,6 +84,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables + - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` ### Usage examples From 04361908e67380cdea7c185b28d8f056330424af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Jul 2023 10:36:02 +0000 Subject: [PATCH 790/871] Fix test_keeper_reconfig_replace_leader --- tests/integration/test_keeper_reconfig_replace_leader/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index ca1ec3a0c92..ef1d5394b67 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -83,6 +83,8 @@ def test_reconfig_replace_leader(started_cluster): assert "node3" in config assert "node4" not in config + ku.wait_configs_equal(config, zk2) + with pytest.raises(Exception): zk1.stop() zk1.close() From 228de12d94a206f6eaae74059216886c32c2b53e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 13:59:23 +0300 Subject: [PATCH 791/871] Update ReplicatedMergeTreeQueue.cpp (#52648) --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e11913fc3d2..21d5597e614 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1803,7 +1803,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo Int64 part_data_version = part->info.getDataVersion(); Int64 part_metadata_version = part->getMetadataVersion(); - LOG_DEBUG(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); + LOG_TEST(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); std::map result; /// Here we return mutation commands for part which has bigger alter version than part metadata version. From 9d73be6fca63edbca0bd97d07386f41268f1b11d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:14:46 +0000 Subject: [PATCH 792/871] test is added --- .../0_stateless/02833_local_udf_options.reference | 1 + tests/queries/0_stateless/02833_local_udf_options.sh | 11 +++++++++++ tests/queries/0_stateless/scripts_udf/function.xml | 9 +++++++++ tests/queries/0_stateless/scripts_udf/udf.sh | 3 +++ 4 files changed, 24 insertions(+) create mode 100755 tests/queries/0_stateless/02833_local_udf_options.reference create mode 100755 tests/queries/0_stateless/02833_local_udf_options.sh create mode 100644 tests/queries/0_stateless/scripts_udf/function.xml create mode 100755 tests/queries/0_stateless/scripts_udf/udf.sh diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference new file mode 100755 index 00000000000..19f0805d8de --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.reference @@ -0,0 +1 @@ +qwerty diff --git a/tests/queries/0_stateless/02833_local_udf_options.sh b/tests/queries/0_stateless/02833_local_udf_options.sh new file mode 100755 index 00000000000..149b62d7e2c --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SCRIPTS_DIR=$CUR_DIR/scripts_udf + +$CLICKHOUSE_LOCAL -q 'select test_function()' -- --user_scripts_path=$SCRIPTS_DIR --user_defined_executable_functions_config=$SCRIPTS_DIR/function.xml diff --git a/tests/queries/0_stateless/scripts_udf/function.xml b/tests/queries/0_stateless/scripts_udf/function.xml new file mode 100644 index 00000000000..69a0abb5cec --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/function.xml @@ -0,0 +1,9 @@ + + + executable + test_function + String + TabSeparated + udf.sh + + diff --git a/tests/queries/0_stateless/scripts_udf/udf.sh b/tests/queries/0_stateless/scripts_udf/udf.sh new file mode 100755 index 00000000000..add85833c3e --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/udf.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +echo qwerty From f3dc6dd061515054afbbe5c58452a9554998a8b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 15:23:57 +0300 Subject: [PATCH 793/871] Revert "Added field `refcount` to `system.remote_data_paths` table" --- src/Disks/IDisk.h | 5 +- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 8 +--- .../02791_remote_paths_refcount.reference | 28 ----------- .../02791_remote_paths_refcount.sql | 47 ------------------- 5 files changed, 4 insertions(+), 86 deletions(-) delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.reference delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.sql diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a2c5e59237f..2b0ca369a96 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -303,11 +303,10 @@ public: std::string local_path; std::string common_prefix_for_objects; StoredObjects objects; - size_t refcount; LocalPathWithObjectStoragePaths( - const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_, size_t refcount_) - : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)), refcount(refcount_) {} + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} }; virtual void getRemotePathsRecursive(const String &, std::vector &) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 001cff4cefe..762151b3808 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -82,7 +82,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path), metadata_storage->getHardlinkCount(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 820b1cf3823..eb514d3b3f4 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -25,7 +25,6 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"size", std::make_shared()}, - {"refcount", std::make_shared()}, {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); @@ -49,7 +48,6 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); MutableColumnPtr col_size = ColumnUInt64::create(); - MutableColumnPtr col_refcount = ColumnUInt64::create(); MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); @@ -67,22 +65,19 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->supportsCache()) cache = FileCacheFactory::instance().getByName(disk->getCacheName()).cache; - for (const auto & [local_path, common_prefox_for_objects, storage_objects, refcount] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { col_disk_name->insert(disk_name); col_base_path->insert(disk->getPath()); - if (cache) col_cache_base_path->insert(cache->getBasePath()); else col_cache_base_path->insertDefault(); - col_local_path->insert(local_path); col_remote_path->insert(object.remote_path); col_size->insert(object.bytes_size); - col_refcount->insert(refcount); col_namespace->insert(common_prefox_for_objects); if (cache) @@ -106,7 +101,6 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); res_columns.emplace_back(std::move(col_size)); - res_columns.emplace_back(std::move(col_refcount)); res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.reference b/tests/queries/0_stateless/02791_remote_paths_refcount.reference deleted file mode 100644 index 56fb1536205..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.reference +++ /dev/null @@ -1,28 +0,0 @@ -0_0_0_0 0 -0_0_0_0_1 1 -1_0_0_0 0 -1_0_0_0_1 1 -0_0_0_0_1 checksums.txt 0 -0_0_0_0_1 columns.txt 1 -0_0_0_0_1 count.txt 1 -0_0_0_0_1 default_compression_codec.txt 1 -0_0_0_0_1 id.bin 1 -0_0_0_0_1 id.cmrk2 1 -0_0_0_0_1 metadata_version.txt 1 -0_0_0_0_1 minmax_id.idx 1 -0_0_0_0_1 partition.dat 1 -0_0_0_0_1 primary.cidx 1 -0_0_0_0_1 v.bin 1 -0_0_0_0_1 v.cmrk2 1 -1_0_0_0_1 checksums.txt 0 -1_0_0_0_1 columns.txt 0 -1_0_0_0_1 count.txt 1 -1_0_0_0_1 default_compression_codec.txt 0 -1_0_0_0_1 id.bin 1 -1_0_0_0_1 id.cmrk2 1 -1_0_0_0_1 metadata_version.txt 0 -1_0_0_0_1 minmax_id.idx 1 -1_0_0_0_1 partition.dat 1 -1_0_0_0_1 primary.cidx 1 -1_0_0_0_1 v.bin 0 -1_0_0_0_1 v.cmrk2 0 diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql deleted file mode 100644 index 180601738ad..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ /dev/null @@ -1,47 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS t_refcount SYNC; - --- Names of parts (on which this test depends) --- can differ in case of fault injection. -SET insert_keeper_fault_injection_probability = 0.0; - -CREATE TABLE t_refcount (id UInt64, v UInt64) -ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') -ORDER BY id PARTITION BY id % 2 -SETTINGS - storage_policy = 's3_cache', - allow_remote_fs_zero_copy_replication = 1, - min_bytes_for_wide_part = 0, - compress_marks = 1, - compress_primary_key = 1, - ratio_of_defaults_for_sparse_serialization = 1.0; - -INSERT INTO t_refcount VALUES (1, 10), (2, 20); - -SET mutations_sync = 2; -ALTER TABLE t_refcount UPDATE v = v * 10 WHERE id % 2 = 1; - -SELECT name, active FROM system.parts WHERE database = currentDatabase() AND table = 't_refcount' ORDER BY name; - -WITH splitByChar('/', full_path) AS path_parts -SELECT path_parts[-2] AS part_name, path_parts[-1] AS file_name, refcount -FROM -( - SELECT - path || local_path AS full_path, - substring(full_path, 1, length(full_path) - position(reverse(full_path), '/') + 1) AS part_path, - refcount - FROM system.remote_data_paths - WHERE disk_name = 's3_cache' -) AS paths -INNER JOIN -( - SELECT path - FROM system.parts - WHERE database = currentDatabase() AND table = 't_refcount' AND active -) AS parts -ON paths.part_path = parts.path -ORDER BY part_name, file_name; - -DROP TABLE IF EXISTS t_refcount SYNC; From 8bba7baeaa65548d91da4c068c6af9b583f9449c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:34:22 +0000 Subject: [PATCH 794/871] fix style --- tests/queries/0_stateless/02833_local_udf_options.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02833_local_udf_options.reference diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference old mode 100755 new mode 100644 From 33300a978e00687713d08fa786178eecc7bc15d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:38:53 +0300 Subject: [PATCH 795/871] Update CHANGELOG (#52655) --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 878edfa4add..f401b346726 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,7 +20,7 @@ * The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). * Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). @@ -115,6 +115,7 @@ * Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). #### Experimental Feature +* Added support for [PRQL](https://prql-lang.org/) as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. * (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). * (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). From 541a253580b968e99c8ba9419ecf82d7459a4768 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 27 Jul 2023 12:43:38 +0000 Subject: [PATCH 796/871] fix crash with function tuple and one sparse argument --- src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.h | 1 + src/Functions/IFunction.cpp | 2 +- .../02833_sparse_columns_tuple_function.reference | 4 ++++ .../02833_sparse_columns_tuple_function.sql | 14 ++++++++++++++ 5 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference create mode 100644 tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index ea05e6ae59b..0bf3f3ac8b3 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -37,6 +37,7 @@ public: bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } + bool canBeInsideSparseColumns() const override { return false; } MutableColumnPtr createColumn() const override; MutableColumnPtr createColumn(const ISerialization & serialization) const override; diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4adafe5d212..54cb3d0d5c2 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -110,6 +110,7 @@ public: /// TODO: support more types. virtual bool supportsSparseSerialization() const { return !haveSubtypes(); } + virtual bool canBeInsideSparseColumns() const { return supportsSparseSerialization(); } SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 650b54d9a37..d119b15733b 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -361,7 +361,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, /// If default of sparse column is changed after execution of function, convert to full column. /// If there are any default in non-zero position after execution of function, convert to full column. /// Currently there is no easy way to rebuild sparse column with new offsets. - if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) + if (!result_type->canBeInsideSparseColumns() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); diff --git a/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference new file mode 100644 index 00000000000..85573e2ed49 --- /dev/null +++ b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference @@ -0,0 +1,4 @@ +(0,0) +(0,0) +(0,1) +(0,NULL) diff --git a/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql new file mode 100644 index 00000000000..776dd35ddba --- /dev/null +++ b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql @@ -0,0 +1,14 @@ +drop table if exists t_tuple_sparse; + +create table t_tuple_sparse (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.0; + +insert into t_tuple_sparse values (0, 0); + +select (a, b) from t_tuple_sparse; +select (a, 0) from t_tuple_sparse; +select (a, 1) from t_tuple_sparse; +select (a, NULL) from t_tuple_sparse; + +drop table if exists t_tuple_sparse; From 78f3a575f9ddbfd47e46e8169b63979e3d2aa72f Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Thu, 27 Jul 2023 21:06:34 +0800 Subject: [PATCH 797/871] Convert hashSets in parallel before merge (#50748) * Convert hashSets in parallel before merge Before merge, if one of the lhs and rhs is singleLevelSet and the other is twoLevelSet, then the SingleLevelSet will call convertToTwoLevel(). The convert process is not in parallel and it will cost lots of cycle if it cosume all the singleLevelSet. The idea of the patch is to convert all the singleLevelSets to twoLevelSets in parallel if the hashsets are not all singleLevel or not all twoLevel. I have tested the patch on Intel 2 x 112 vCPUs SPR server with clickbench and latest upstream ClickHouse. Q5 has got a big 264% performance improvement and 24 queries have got at least 5% performance gain. The overall geomean of 43 queries has gained 7.4% more than the base code. Signed-off-by: Jiebin Sun * add resize() for the data_vec in parallelizeMergePrepare() Signed-off-by: Jiebin Sun * Add the performance test prepare_hash_before_merge.xml Signed-off-by: Jiebin Sun * Fit the CI to rename the data set from hits_v1 to test.hits. Signed-off-by: Jiebin Sun * remove the redundant branch in UniqExactSet Co-authored-by: Nikita Taranov * Remove the empty methods and add throw exception in parallelizeMergePrepare() Signed-off-by: Jiebin Sun --------- Signed-off-by: Jiebin Sun Co-authored-by: Nikita Taranov --- .../AggregateFunctionUniq.h | 39 ++++++++++++++ src/AggregateFunctions/IAggregateFunction.h | 8 +++ src/AggregateFunctions/UniqExactSet.h | 51 +++++++++++++++++++ src/Interpreters/Aggregator.cpp | 14 +++++ .../performance/prepare_hash_before_merge.xml | 4 ++ 5 files changed, 116 insertions(+) create mode 100644 tests/performance/prepare_hash_before_merge.xml diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..2810051a82f 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -29,6 +29,10 @@ #include #include +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} namespace DB { @@ -42,6 +46,7 @@ struct AggregateFunctionUniqUniquesHashSetData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniq"; } @@ -55,6 +60,7 @@ struct AggregateFunctionUniqUniquesHashSetDataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -72,6 +78,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -84,6 +91,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -96,6 +104,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -108,6 +117,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -120,6 +130,7 @@ struct AggregateFunctionUniqHLL12DataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -143,6 +154,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -162,6 +174,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -181,6 +194,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -190,6 +204,7 @@ template { constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -204,6 +219,7 @@ struct AggregateFunctionUniqThetaData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqTheta"; } @@ -213,6 +229,7 @@ template struct AggregateFunctionUniqThetaDataForVariadic : AggregateFunctionUniqThetaData { constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -384,8 +401,10 @@ template class AggregateFunctionUniq final : public IAggregateFunctionDataHelper> { private: + using DataSet = typename Data::Set; static constexpr size_t num_args = 1; static constexpr bool is_able_to_parallelize_merge = Data::is_able_to_parallelize_merge; + static constexpr bool is_parallelize_merge_prepare_needed = Data::is_parallelize_merge_prepare_needed; public: explicit AggregateFunctionUniq(const DataTypes & argument_types_) @@ -439,6 +458,26 @@ public: detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, null_map); } + bool isParallelizeMergePrepareNeeded() const override { return is_parallelize_merge_prepare_needed;} + + void parallelizeMergePrepare(AggregateDataPtrs & places, ThreadPool & thread_pool) const override + { + if constexpr (is_parallelize_merge_prepare_needed) + { + std::vector data_vec; + data_vec.resize(places.size()); + + for (unsigned long i = 0; i < data_vec.size(); i++) + data_vec[i] = &this->data(places[i]).set; + + DataSet::parallelizeMergePrepare(data_vec, thread_pool); + } + else + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() is only implemented when is_parallelize_merge_prepare_needed is true for {} ", getName()); + } + } + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).set.merge(this->data(rhs).set); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index df08b6f2109..b460a66ea22 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -47,6 +47,7 @@ using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; using AggregateDataPtr = char *; +using AggregateDataPtrs = std::vector; using ConstAggregateDataPtr = const char *; class IAggregateFunction; @@ -148,6 +149,13 @@ public: /// Default values must be a the 0-th positions in columns. virtual void addManyDefaults(AggregateDataPtr __restrict place, const IColumn ** columns, size_t length, Arena * arena) const = 0; + virtual bool isParallelizeMergePrepareNeeded() const { return false; } + + virtual void parallelizeMergePrepare(AggregateDataPtrs & /*places*/, ThreadPool & /*thread_pool*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() with thread pool parameter isn't implemented for {} ", getName()); + } + /// Merges state (on which place points to) with other state of current aggregation function. virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0; diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 90cfe700179..0d99b29686f 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -28,6 +28,57 @@ public: asTwoLevel().insert(std::forward(arg)); } + /// In merge, if one of the lhs and rhs is twolevelset and the other is singlelevelset, then the singlelevelset will need to convertToTwoLevel(). + /// It's not in parallel and will cost extra large time if the thread_num is large. + /// This method will convert all the SingleLevelSet to TwoLevelSet in parallel if the hashsets are not all singlelevel or not all twolevel. + static void parallelizeMergePrepare(const std::vector & data_vec, ThreadPool & thread_pool) + { + unsigned long single_level_set_num = 0; + + for (auto ele : data_vec) + { + if (ele->isSingleLevel()) + single_level_set_num ++; + } + + if (single_level_set_num > 0 && single_level_set_num < data_vec.size()) + { + try + { + auto data_vec_atomic_index = std::make_shared(0); + auto thread_func = [data_vec, data_vec_atomic_index, thread_group = CurrentThread::getGroup()]() + { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + + setThreadName("UniqExaConvert"); + + while (true) + { + const auto i = data_vec_atomic_index->fetch_add(1); + if (i >= data_vec.size()) + return; + if (data_vec[i]->isSingleLevel()) + data_vec[i]->convertToTwoLevel(); + } + }; + for (size_t i = 0; i < std::min(thread_pool.getMaxThreads(), single_level_set_num); ++i) + thread_pool.scheduleOrThrowOnError(thread_func); + + thread_pool.wait(); + } + catch (...) + { + thread_pool.wait(); + throw; + } + } + } + auto merge(const UniqExactSet & other, ThreadPool * thread_pool = nullptr) { if (isSingleLevel() && other.isTwoLevel()) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 36cd32910b5..c2914c938b5 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2603,6 +2603,20 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( AggregatedDataVariantsPtr & res = non_empty_data[0]; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (aggregate_functions[i]->isParallelizeMergePrepareNeeded()) + { + size_t size = non_empty_data.size(); + std::vector data_vec; + + for (size_t result_num = 0; result_num < size; ++result_num) + data_vec.emplace_back(non_empty_data[result_num]->without_key + offsets_of_aggregate_states[i]); + + aggregate_functions[i]->parallelizeMergePrepare(data_vec, thread_pool); + } + } + /// We merge all aggregation results to the first. for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) { diff --git a/tests/performance/prepare_hash_before_merge.xml b/tests/performance/prepare_hash_before_merge.xml new file mode 100644 index 00000000000..e99f762927f --- /dev/null +++ b/tests/performance/prepare_hash_before_merge.xml @@ -0,0 +1,4 @@ + + SELECT COUNT(DISTINCT Title) FROM test.hits SETTINGS max_threads = 24 + SELECT COUNT(DISTINCT Referer) FROM test.hits SETTINGS max_threads = 22 + From 671128140dc6672349421c84643076410d46ce0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:34:05 +0200 Subject: [PATCH 798/871] Update autogenerated version to 23.8.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 ++++---- .../StorageSystemContributors.generated.cpp | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 821b7b46855..9919d018046 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54476) +SET(VERSION_REVISION 54477) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 7) +SET(VERSION_MINOR 8) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH d1c7e13d08868cb04d3562dcced704dd577cb1df) -SET(VERSION_DESCRIBE v23.7.1.1-testing) -SET(VERSION_STRING 23.7.1.1) +SET(VERSION_GITHASH a70127baecc451f1f7073bad7b6198f6703441d8) +SET(VERSION_DESCRIBE v23.8.1.1-testing) +SET(VERSION_STRING 23.8.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f84c554afc0..031c7454ab6 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -42,10 +42,12 @@ const char * auto_contributors[] { "Alex", "Alex Bocharov", "Alex Cao", + "Alex Cheng", "Alex Karo", "Alex Krash", "Alex Ryndin", "Alex Zatelepin", + "AlexBykovski", "Alexander Avdonkin", "Alexander Bezpiatov", "Alexander Burmak", @@ -232,6 +234,7 @@ const char * auto_contributors[] { "CheSema", "Chebarykov Pavel", "Chen Yufei", + "Chen768959", "Cheng Pan", "Chienlung Cheung", "Christian", @@ -485,6 +488,7 @@ const char * auto_contributors[] { "John", "John Hummel", "John Skopis", + "John Spurlock", "Jonatas Freitas", "Jonathan-Ackerman", "Jordi", @@ -659,6 +663,7 @@ const char * auto_contributors[] { "Mikhail Gaidamaka", "Mikhail Guzov", "Mikhail Korotov", + "Mikhail Koviazin", "Mikhail Malafeev", "Mikhail Nacharov", "Mikhail Salosin", @@ -815,6 +820,7 @@ const char * auto_contributors[] { "Roman Vasin", "Roman Vlasenko", "Roman Zhukov", + "Rory Crispin", "Roy Bellingan", "Ruslan", "Ruslan Savchenko", @@ -832,7 +838,9 @@ const char * auto_contributors[] { "Salvatore Mesoraca", "Sami Kerola", "Samuel Chou", + "Samuel Colvin", "San", + "Sanjam Panda", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -883,6 +891,7 @@ const char * auto_contributors[] { "SmitaRKulkarni", "Snow", "Sofia Antipushina", + "Song Liyong", "Sorck", "Stanislav Dobrovolschii", "Stanislav Pavlovichev", @@ -893,6 +902,7 @@ const char * auto_contributors[] { "Stepan Herold", "Stephan", "Steve-金勇", + "StianBerger", "Stig Bakken", "Storozhuk Kostiantyn", "Stupnikov Andrey", @@ -977,6 +987,7 @@ const char * auto_contributors[] { "Vitaliy Karnienko", "Vitaliy Kozlovskiy", "Vitaliy Lyudvichenko", + "Vitaliy Pashkov", "Vitaliy Zakaznikov", "Vitaly", "Vitaly Artemyev", @@ -1029,6 +1040,7 @@ const char * auto_contributors[] { "Yakov Olkhovskiy", "YalalovSM", "Yangkuan Liu", + "Yarik Briukhovetskyi", "Yatian Xu", "Yatsishin Ilya", "Yağızcan Değirmenci", @@ -1053,6 +1065,7 @@ const char * auto_contributors[] { "Yury Karpovich", "Yury Stankevich", "Yusuke Tanaka", + "Zach Naimon", "ZhiYong Wang", "Zhichang Yu", "Zhichun Wu", @@ -1143,6 +1156,7 @@ const char * auto_contributors[] { "changvvb", "chasingegg", "chen", + "chen768959", "chen9t", "chengy8934", "chenjian", @@ -1179,6 +1193,7 @@ const char * auto_contributors[] { "detailyang", "dfenelonov", "dgrr", + "dheerajathrey", "dimarub2000", "dinosaur", "divanorama", @@ -1329,6 +1344,7 @@ const char * auto_contributors[] { "lanfz", "larryluogit", "laurieliyang", + "lcjh", "lehasm", "leosunli", "leozhang", @@ -1455,6 +1471,7 @@ const char * auto_contributors[] { "pawelsz-rb", "pdai", "pdv-ru", + "pedro.riera", "pengxiangcai", "peshkurov", "peter279k", @@ -1548,8 +1565,10 @@ const char * auto_contributors[] { "teng.ma", "terrylin", "tesw yew isal", + "therealnick233", "tianzhou", "tiger.yan", + "timfursov", "tison", "topvisor", "tpanetti", @@ -1563,6 +1582,7 @@ const char * auto_contributors[] { "usurai", "vahid-sohrabloo", "vdimir", + "velavokr", "velom", "vesslanjin", "vgocoder", @@ -1587,17 +1607,21 @@ const char * auto_contributors[] { "wuxiaobai24", "wzl", "xPoSx", + "xiao", + "xiaolei565", "xiedeyantu", "xieyichen", "xinhuitian", "xlwh", "xmy", + "xuelei", "yakkomajuri", "yakov-olkhovskiy", "yandd", "yang", "yangshuai", "yaqi-zhao", + "yariks5s", "yeer", "ygrek", "yhgcn", From 28c49e3f20923917c46872b15dce21a72ab47b4a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 13:47:11 +0000 Subject: [PATCH 799/871] Update version_date.tsv and changelogs after v23.7.1.2470-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.7.1.2470-stable.md | 452 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 458 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v23.7.1.2470-stable.md diff --git a/SECURITY.md b/SECURITY.md index 4ba5f13d09c..d61533b44b9 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.7 | ✔️ | | 23.6 | ✔️ | | 23.5 | ✔️ | -| 23.4 | ✔️ | +| 23.4 | ❌ | | 23.3 | ✔️ | | 23.2 | ❌ | | 23.1 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 8a6324aef88..c9800e4e66d 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 7f453627601..f558338b23c 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 1fa7b83ae16..156de034a7f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.7.1.2470-stable.md b/docs/changelogs/v23.7.1.2470-stable.md new file mode 100644 index 00000000000..a77078cb653 --- /dev/null +++ b/docs/changelogs/v23.7.1.2470-stable.md @@ -0,0 +1,452 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.7.1.2470-stable (a70127baecc) FIXME as compared to v23.6.1.1524-stable (d1c7e13d088) + +#### Backward Incompatible Change +* Add ` NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after **downgrade** (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The function `toDecimalString` is removed due to subpar implementation quality. This closes [#52407](https://github.com/ClickHouse/ClickHouse/issues/52407). [#52450](https://github.com/ClickHouse/ClickHouse/pull/52450) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Implement KQL-style formatting for Interval. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Support ZooKeeper `reconfig` command for CH Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added support for prql as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add a column is_obsolete to system.settings and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Just a new request of [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add SYSTEM STOP LISTEN query. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add input_format_csv_allow_variable_number_of_columns options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function substring_index, as in spark or mysql. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* Show stats for jemalloc bins. Example ``` SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add RowBinaryWithDefaults format with extra byte before each column for using column default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new initcap / initcapUTF8 functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage (for query) to client final statistics, and to http header. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new hasSubsequence() (+CaseInsensitive + UTF8 versions) functions. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Re-add SipHash keyed functions. [#52206](https://github.com/ClickHouse/ClickHouse/pull/52206) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). + +#### Performance Improvement +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Remove duplicate condition in functionunixtimestamp64.h. [#51857](https://github.com/ClickHouse/ClickHouse/pull/51857) ([lcjh](https://github.com/ljhcage)). +* The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* 1. Add rewriter for both old and new analyzer. 2. Add settings `optimize_uniq_to_count` which default is 0. [#52004](https://github.com/ClickHouse/ClickHouse/pull/52004) ([JackyWoo](https://github.com/JackyWoo)). +* The performance experiments of **OnTime** on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of **11.6%** to the QPS of the query **Q8** while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1` . This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823) . This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173) . [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in FileCache::loadMetadata. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). + +#### Improvement +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues: ``` select geohashEncode(120.2, number::Float64) from numbers(10);. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow filtering HTTP headers with `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* #50727 new alias for function current_database and added new function current_schemas. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Log async insert flush queries into to system.query_log. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Fix checking error caused by uninitialized class members. [#51418](https://github.com/ClickHouse/ClickHouse/pull/51418) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Correcting the message of modify storage policy https://github.com/clickhouse/clickhouse/issues/51516 ### documentation entry for user-facing changes. [#51519](https://github.com/ClickHouse/ClickHouse/pull/51519) ([xiaolei565](https://github.com/xiaolei565)). +* Support `DROP FILESYSTEM CACHE KEY [ OFFSET ]`. [#51547](https://github.com/ClickHouse/ClickHouse/pull/51547) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column modification time into system.parts_columns. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function range of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of statement SHOW INDEX. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* CacheDictionary request only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed settings not applied for explain query when format provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Fixed crash when mysqlxx::Pool::Entry is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* CREATE TABLE ... AS SELECT .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Log messages are written to text_log from the beginning. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception will be thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Support async_deduplication_token for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Avro input format support Union with single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). This is defaulted to false until [#52075](https://github.com/ClickHouse/ClickHouse/issues/52075) is fixed. [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* 1. Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#52181](https://github.com/ClickHouse/ClickHouse/pull/52181) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now unquoted utf-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093) . [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now it's possible to specify min (`memory_profiler_sample_min_allocation_size`) and max (`memory_profiler_sample_max_allocation_size`) size for allocations to be tracked with sampling memory profiler. [#52419](https://github.com/ClickHouse/ClickHouse/pull/52419) ([alesapin](https://github.com/alesapin)). +* The `session_timezone` setting is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). +* Added field `refcount` to `system.remote_data_paths` table. [#52518](https://github.com/ClickHouse/ClickHouse/pull/52518) ([Anton Popov](https://github.com/CurtizJ)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed CRC32(WeakHash32) issue for s390x. [#50365](https://github.com/ClickHouse/ClickHouse/pull/50365) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Add integration test check with the enabled analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) ([Dmitry Novik](https://github.com/novikd)). +* Update cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed several issues found by OSS-Fuzz. [#51736](https://github.com/ClickHouse/ClickHouse/pull/51736) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* There were a couple of failures because of (?) S3 availability. The sccache has a feature of failing over to local compilation. [#51893](https://github.com/ClickHouse/ClickHouse/pull/51893) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* 02242_delete_user_race and 02243_drop_user_grant_race tests have been corrected. [#51923](https://github.com/ClickHouse/ClickHouse/pull/51923) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Follow up [#50926](https://github.com/ClickHouse/ClickHouse/issues/50926). Add integration tests check with enabled analyzer to master. [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Improve the startup time of `clickhouse-client` and `clickhouse-local` in debug and sanitizer builds. This closes [#52228](https://github.com/ClickHouse/ClickHouse/issues/52228). [#52489](https://github.com/ClickHouse/ClickHouse/pull/52489) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix materialised pg syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add documentation for building in docker"'. [#51773](https://github.com/ClickHouse/ClickHouse/pull/51773) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix build"'. [#51911](https://github.com/ClickHouse/ClickHouse/pull/51911) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Millisecond and microsecond support in date_diff / age functions"'. [#52129](https://github.com/ClickHouse/ClickHouse/pull/52129) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Re-add SipHash keyed functions"'. [#52466](https://github.com/ClickHouse/ClickHouse/pull/52466) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Add an ability to specify allocations size for sampling memory profiler"'. [#52496](https://github.com/ClickHouse/ClickHouse/pull/52496) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Rewrite uniq to count"'. [#52576](https://github.com/ClickHouse/ClickHouse/pull/52576) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Remove duplicate_order_by_and_distinct optimization [#47135](https://github.com/ClickHouse/ClickHouse/pull/47135) ([Igor Nikonov](https://github.com/devcrafter)). +* Update sort desc in ReadFromMergeTree after applying PREWHERE info [#48669](https://github.com/ClickHouse/ClickHouse/pull/48669) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `BindException: Address already in use` in HDFS integration tests [#49428](https://github.com/ClickHouse/ClickHouse/pull/49428) ([Nikita Taranov](https://github.com/nickitat)). +* Force libunwind usage (removes gcc_eh support) [#49438](https://github.com/ClickHouse/ClickHouse/pull/49438) ([Azat Khuzhin](https://github.com/azat)). +* Cleanup `storage_conf.xml` [#49557](https://github.com/ClickHouse/ClickHouse/pull/49557) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky tests caused by OPTIMIZE FINAL failing memory budget check [#49764](https://github.com/ClickHouse/ClickHouse/pull/49764) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove unstable queries from performance/join_set_filter [#50235](https://github.com/ClickHouse/ClickHouse/pull/50235) ([vdimir](https://github.com/vdimir)). +* More accurate DNS resolve for the keeper connection [#50738](https://github.com/ClickHouse/ClickHouse/pull/50738) ([pufit](https://github.com/pufit)). +* Try to fix some trash in Disks and part moves [#51135](https://github.com/ClickHouse/ClickHouse/pull/51135) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add jemalloc support fro s390x [#51186](https://github.com/ClickHouse/ClickHouse/pull/51186) ([Boris Kuschel](https://github.com/bkuschel)). +* Resubmit [#48821](https://github.com/ClickHouse/ClickHouse/issues/48821) [#51208](https://github.com/ClickHouse/ClickHouse/pull/51208) ([Kseniia Sumarokova](https://github.com/kssenii)). +* test for [#36894](https://github.com/ClickHouse/ClickHouse/issues/36894) [#51274](https://github.com/ClickHouse/ClickHouse/pull/51274) ([Denny Crane](https://github.com/den-crane)). +* external_aggregation_fix for big endian machines [#51280](https://github.com/ClickHouse/ClickHouse/pull/51280) ([Sanjam Panda](https://github.com/saitama951)). +* Fix: Invalid number of rows in Chunk column Object [#51296](https://github.com/ClickHouse/ClickHouse/pull/51296) ([Igor Nikonov](https://github.com/devcrafter)). +* Add a test for [#44816](https://github.com/ClickHouse/ClickHouse/issues/44816) [#51305](https://github.com/ClickHouse/ClickHouse/pull/51305) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for `calculate_text_stack_trace` setting [#51311](https://github.com/ClickHouse/ClickHouse/pull/51311) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* decrease log level, make logs shorter [#51320](https://github.com/ClickHouse/ClickHouse/pull/51320) ([Sema Checherinda](https://github.com/CheSema)). +* Collect stack traces from job's scheduling and print along with exception's stack trace. [#51349](https://github.com/ClickHouse/ClickHouse/pull/51349) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add a test for [#42691](https://github.com/ClickHouse/ClickHouse/issues/42691) [#51352](https://github.com/ClickHouse/ClickHouse/pull/51352) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#32474](https://github.com/ClickHouse/ClickHouse/issues/32474) [#51354](https://github.com/ClickHouse/ClickHouse/pull/51354) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#41727](https://github.com/ClickHouse/ClickHouse/issues/41727) [#51355](https://github.com/ClickHouse/ClickHouse/pull/51355) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#35801](https://github.com/ClickHouse/ClickHouse/issues/35801) [#51356](https://github.com/ClickHouse/ClickHouse/pull/51356) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#34626](https://github.com/ClickHouse/ClickHouse/issues/34626) [#51357](https://github.com/ClickHouse/ClickHouse/pull/51357) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Initialize text_log earlier to capture table startup messages [#51360](https://github.com/ClickHouse/ClickHouse/pull/51360) ([Azat Khuzhin](https://github.com/azat)). +* Use separate default settings for clickhouse-local [#51363](https://github.com/ClickHouse/ClickHouse/pull/51363) ([Azat Khuzhin](https://github.com/azat)). +* Attempt to remove wrong code (catch/throw in Functions) [#51367](https://github.com/ClickHouse/ClickHouse/pull/51367) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove suspicious code [#51383](https://github.com/ClickHouse/ClickHouse/pull/51383) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable hedged requests under TSan [#51392](https://github.com/ClickHouse/ClickHouse/pull/51392) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* no finalize in d-tor WriteBufferFromOStream [#51404](https://github.com/ClickHouse/ClickHouse/pull/51404) ([Sema Checherinda](https://github.com/CheSema)). +* Better diagnostics for 01193_metadata_loading [#51414](https://github.com/ClickHouse/ClickHouse/pull/51414) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix attaching gdb in stress tests [#51445](https://github.com/ClickHouse/ClickHouse/pull/51445) ([Kruglov Pavel](https://github.com/Avogar)). +* Merging [#36384](https://github.com/ClickHouse/ClickHouse/issues/36384) [#51458](https://github.com/ClickHouse/ClickHouse/pull/51458) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible race on shutdown wait [#51497](https://github.com/ClickHouse/ClickHouse/pull/51497) ([Sergei Trifonov](https://github.com/serxa)). +* Fix `test_alter_moving_garbage`: lock between getActiveContainingPart and swapActivePart in parts mover [#51498](https://github.com/ClickHouse/ClickHouse/pull/51498) ([vdimir](https://github.com/vdimir)). +* Fix a logical error on mutation [#51502](https://github.com/ClickHouse/ClickHouse/pull/51502) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix running integration tests with spaces in it's names [#51514](https://github.com/ClickHouse/ClickHouse/pull/51514) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky test 00417_kill_query [#51522](https://github.com/ClickHouse/ClickHouse/pull/51522) ([Nikolay Degterinsky](https://github.com/evillique)). +* fs cache: add some checks [#51536](https://github.com/ClickHouse/ClickHouse/pull/51536) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't run 02782_uniq_exact_parallel_merging_bug in parallel with other tests [#51549](https://github.com/ClickHouse/ClickHouse/pull/51549) ([Nikita Taranov](https://github.com/nickitat)). +* 00900_orc_load: lift kill timeout [#51559](https://github.com/ClickHouse/ClickHouse/pull/51559) ([Robert Schulze](https://github.com/rschu1ze)). +* Add retries to 00416_pocopatch_progress_in_http_headers [#51575](https://github.com/ClickHouse/ClickHouse/pull/51575) ([Nikolay Degterinsky](https://github.com/evillique)). +* Remove the usage of Analyzer setting in the client [#51578](https://github.com/ClickHouse/ClickHouse/pull/51578) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix merge_selecting_task scheduling [#51591](https://github.com/ClickHouse/ClickHouse/pull/51591) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add hex functions for cityhash [#51595](https://github.com/ClickHouse/ClickHouse/pull/51595) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove `unset CLICKHOUSE_LOG_COMMENT` from tests [#51623](https://github.com/ClickHouse/ClickHouse/pull/51623) ([Nikita Taranov](https://github.com/nickitat)). +* Implement endianness-independent serialization [#51637](https://github.com/ClickHouse/ClickHouse/pull/51637) ([ltrk2](https://github.com/ltrk2)). +* Ignore APPEND and TRUNCATE modifiers if file does not exist. [#51640](https://github.com/ClickHouse/ClickHouse/pull/51640) ([alekar](https://github.com/alekar)). +* Try to fix flaky 02210_processors_profile_log [#51641](https://github.com/ClickHouse/ClickHouse/pull/51641) ([Igor Nikonov](https://github.com/devcrafter)). +* Make common macros extendable [#51646](https://github.com/ClickHouse/ClickHouse/pull/51646) ([Amos Bird](https://github.com/amosbird)). +* Correct an exception message in src/Functions/nested.cpp [#51651](https://github.com/ClickHouse/ClickHouse/pull/51651) ([Alex Cheng](https://github.com/Alex-Cheng)). +* tests: fix 02050_client_profile_events flakiness [#51653](https://github.com/ClickHouse/ClickHouse/pull/51653) ([Azat Khuzhin](https://github.com/azat)). +* Minor follow-up to re2 update to 2023-06-02 ([#50949](https://github.com/ClickHouse/ClickHouse/issues/50949)) [#51655](https://github.com/ClickHouse/ClickHouse/pull/51655) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 02116_tuple_element with Analyzer [#51669](https://github.com/ClickHouse/ClickHouse/pull/51669) ([Robert Schulze](https://github.com/rschu1ze)). +* Update timeouts in tests for transactions [#51683](https://github.com/ClickHouse/ClickHouse/pull/51683) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused code [#51684](https://github.com/ClickHouse/ClickHouse/pull/51684) ([Sergei Trifonov](https://github.com/serxa)). +* Remove `mmap/mremap/munmap` from Allocator.h [#51686](https://github.com/ClickHouse/ClickHouse/pull/51686) ([alesapin](https://github.com/alesapin)). +* SonarCloud: Add C++23 Experimental Flag [#51687](https://github.com/ClickHouse/ClickHouse/pull/51687) ([Julio Jimenez](https://github.com/juliojimenez)). +* Wait with retries when attaching GDB in tests [#51688](https://github.com/ClickHouse/ClickHouse/pull/51688) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v23.6.1.1524-stable [#51691](https://github.com/ClickHouse/ClickHouse/pull/51691) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* fix write to finalized buffer [#51696](https://github.com/ClickHouse/ClickHouse/pull/51696) ([Sema Checherinda](https://github.com/CheSema)). +* do not log exception aborted for pending mutate/merge entries when shutdown [#51697](https://github.com/ClickHouse/ClickHouse/pull/51697) ([Sema Checherinda](https://github.com/CheSema)). +* Fix race in ContextAccess [#51704](https://github.com/ClickHouse/ClickHouse/pull/51704) ([Vitaly Baranov](https://github.com/vitlibar)). +* Make test scripts backwards compatible [#51707](https://github.com/ClickHouse/ClickHouse/pull/51707) ([Antonio Andelic](https://github.com/antonio2368)). +* test for full join and null predicate [#51709](https://github.com/ClickHouse/ClickHouse/pull/51709) ([Denny Crane](https://github.com/den-crane)). +* A cmake warning on job limits underutilizing CPU [#51710](https://github.com/ClickHouse/ClickHouse/pull/51710) ([velavokr](https://github.com/velavokr)). +* Fix SQLLogic docker images [#51719](https://github.com/ClickHouse/ClickHouse/pull/51719) ([Antonio Andelic](https://github.com/antonio2368)). +* Added ASK_PASSWORD client constant instead of hardcoded '\n' [#51723](https://github.com/ClickHouse/ClickHouse/pull/51723) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Update README.md [#51726](https://github.com/ClickHouse/ClickHouse/pull/51726) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix source image for sqllogic [#51728](https://github.com/ClickHouse/ClickHouse/pull/51728) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove MemoryPool from Poco because it's useless [#51732](https://github.com/ClickHouse/ClickHouse/pull/51732) ([alesapin](https://github.com/alesapin)). +* Fix: logical error in grace hash join [#51737](https://github.com/ClickHouse/ClickHouse/pull/51737) ([Igor Nikonov](https://github.com/devcrafter)). +* Update 01320_create_sync_race_condition_zookeeper.sh [#51742](https://github.com/ClickHouse/ClickHouse/pull/51742) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Pin for docker-ce [#51743](https://github.com/ClickHouse/ClickHouse/pull/51743) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert "Fix: Invalid number of rows in Chunk column Object" [#51750](https://github.com/ClickHouse/ClickHouse/pull/51750) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add SonarCloud to README [#51751](https://github.com/ClickHouse/ClickHouse/pull/51751) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix test `02789_object_type_invalid_num_of_rows` [#51754](https://github.com/ClickHouse/ClickHouse/pull/51754) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix (benign) data race in `transform` [#51755](https://github.com/ClickHouse/ClickHouse/pull/51755) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky KeeperMap test [#51764](https://github.com/ClickHouse/ClickHouse/pull/51764) ([Antonio Andelic](https://github.com/antonio2368)). +* Version mypy=1.4.1 falsly reports unused ignore comment [#51769](https://github.com/ClickHouse/ClickHouse/pull/51769) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid keeping lock Context::getLock() while calculating access rights [#51772](https://github.com/ClickHouse/ClickHouse/pull/51772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Making stateless tests with timeout less flaky [#51774](https://github.com/ClickHouse/ClickHouse/pull/51774) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix after [#51000](https://github.com/ClickHouse/ClickHouse/issues/51000) [#51790](https://github.com/ClickHouse/ClickHouse/pull/51790) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add assert in ThreadStatus destructor for correct current_thread [#51800](https://github.com/ClickHouse/ClickHouse/pull/51800) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix broken parts handling in `ReplicatedMergeTree` [#51801](https://github.com/ClickHouse/ClickHouse/pull/51801) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix tsan signal-unsafe call [#51802](https://github.com/ClickHouse/ClickHouse/pull/51802) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix for parallel replicas not completely disabled by granule count threshold [#51805](https://github.com/ClickHouse/ClickHouse/pull/51805) ([Alexander Gololobov](https://github.com/davenger)). +* Make sure that we don't attempt to serialize/deserialize block with 0 columns and non-zero rows [#51807](https://github.com/ClickHouse/ClickHouse/pull/51807) ([Alexander Gololobov](https://github.com/davenger)). +* Fix rare bug in `DROP COLUMN` and enabled sparse columns [#51809](https://github.com/ClickHouse/ClickHouse/pull/51809) ([Anton Popov](https://github.com/CurtizJ)). +* Fix flaky `test_multiple_disks` [#51821](https://github.com/ClickHouse/ClickHouse/pull/51821) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow up to [#51547](https://github.com/ClickHouse/ClickHouse/issues/51547) [#51822](https://github.com/ClickHouse/ClickHouse/pull/51822) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Correctly grep archives in stress tests [#51824](https://github.com/ClickHouse/ClickHouse/pull/51824) ([Antonio Andelic](https://github.com/antonio2368)). +* Update analyzer_tech_debt.txt [#51836](https://github.com/ClickHouse/ClickHouse/pull/51836) ([Alexander Tokmakov](https://github.com/tavplubix)). +* remove unused code [#51837](https://github.com/ClickHouse/ClickHouse/pull/51837) ([flynn](https://github.com/ucasfl)). +* Fix disk config for upgrade tests [#51839](https://github.com/ClickHouse/ClickHouse/pull/51839) ([Antonio Andelic](https://github.com/antonio2368)). +* Remove Coverity from workflows, but leave in the code [#51842](https://github.com/ClickHouse/ClickHouse/pull/51842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Many fixes [3] [#51848](https://github.com/ClickHouse/ClickHouse/pull/51848) ([Ilya Yatsishin](https://github.com/qoega)). +* Change misleading name in joins: addJoinedBlock -> addBlockToJoin [#51852](https://github.com/ClickHouse/ClickHouse/pull/51852) ([Igor Nikonov](https://github.com/devcrafter)). +* fix: correct exception messages on policies comparison [#51854](https://github.com/ClickHouse/ClickHouse/pull/51854) ([Feng Kaiyu](https://github.com/fky2015)). +* Update 02439_merge_selecting_partitions.sql [#51862](https://github.com/ClickHouse/ClickHouse/pull/51862) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove useless packages [#51863](https://github.com/ClickHouse/ClickHouse/pull/51863) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless logs [#51865](https://github.com/ClickHouse/ClickHouse/pull/51865) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect log level = warning [#51867](https://github.com/ClickHouse/ClickHouse/pull/51867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_replicated_table_attach [#51868](https://github.com/ClickHouse/ClickHouse/pull/51868) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better usability of a test [#51869](https://github.com/ClickHouse/ClickHouse/pull/51869) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless code [#51873](https://github.com/ClickHouse/ClickHouse/pull/51873) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix upgrade check script [#51878](https://github.com/ClickHouse/ClickHouse/pull/51878) ([Antonio Andelic](https://github.com/antonio2368)). +* Sqlloogic improvements [#51883](https://github.com/ClickHouse/ClickHouse/pull/51883) ([Ilya Yatsishin](https://github.com/qoega)). +* Disable ThinLTO on non-Linux [#51897](https://github.com/ClickHouse/ClickHouse/pull/51897) ([Robert Schulze](https://github.com/rschu1ze)). +* Pin rust nightly (to make it stable) [#51903](https://github.com/ClickHouse/ClickHouse/pull/51903) ([Azat Khuzhin](https://github.com/azat)). +* Fix build [#51909](https://github.com/ClickHouse/ClickHouse/pull/51909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix build [#51910](https://github.com/ClickHouse/ClickHouse/pull/51910) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `00175_partition_by_ignore` and move it to correct location [#51913](https://github.com/ClickHouse/ClickHouse/pull/51913) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test 02360_send_logs_level_colors: avoid usage of `file` tool [#51914](https://github.com/ClickHouse/ClickHouse/pull/51914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Maybe better tests [#51916](https://github.com/ClickHouse/ClickHouse/pull/51916) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert system drop filesystem cache by key [#51917](https://github.com/ClickHouse/ClickHouse/pull/51917) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `detach_attach_partition_race` [#51920](https://github.com/ClickHouse/ClickHouse/pull/51920) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix for `02481_async_insert_race_long` [#51925](https://github.com/ClickHouse/ClickHouse/pull/51925) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix segfault caused by `ThreadStatus` [#51931](https://github.com/ClickHouse/ClickHouse/pull/51931) ([Antonio Andelic](https://github.com/antonio2368)). +* Print short fault info only from safe fields [#51932](https://github.com/ClickHouse/ClickHouse/pull/51932) ([Alexander Gololobov](https://github.com/davenger)). +* Fix typo in integration tests [#51944](https://github.com/ClickHouse/ClickHouse/pull/51944) ([Ilya Yatsishin](https://github.com/qoega)). +* Better logs on shutdown [#51951](https://github.com/ClickHouse/ClickHouse/pull/51951) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Filter databases list before querying potentially slow fields [#51955](https://github.com/ClickHouse/ClickHouse/pull/51955) ([Alexander Gololobov](https://github.com/davenger)). +* Fix some issues with transactions [#51959](https://github.com/ClickHouse/ClickHouse/pull/51959) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix unrelated messages from LSan in clickhouse-client [#51966](https://github.com/ClickHouse/ClickHouse/pull/51966) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow OOM in AST Fuzzer with Sanitizers [#51967](https://github.com/ClickHouse/ClickHouse/pull/51967) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable one test under Analyzer [#51968](https://github.com/ClickHouse/ClickHouse/pull/51968) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Docker [#51969](https://github.com/ClickHouse/ClickHouse/pull/51969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `01825_type_json_from_map` [#51970](https://github.com/ClickHouse/ClickHouse/pull/51970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02354_distributed_with_external_aggregation_memory_usage` [#51971](https://github.com/ClickHouse/ClickHouse/pull/51971) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix disaster in integration tests, part 2 [#51973](https://github.com/ClickHouse/ClickHouse/pull/51973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [RFC] Cleanup remote_servers in dist config.xml [#51985](https://github.com/ClickHouse/ClickHouse/pull/51985) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.6.2.18-stable [#51986](https://github.com/ClickHouse/ClickHouse/pull/51986) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.20.11-lts [#51987](https://github.com/ClickHouse/ClickHouse/pull/51987) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix performance test for regexp cache [#51988](https://github.com/ClickHouse/ClickHouse/pull/51988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Move a test to the right place [#51989](https://github.com/ClickHouse/ClickHouse/pull/51989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a check to validate that the stateful tests are stateful [#51990](https://github.com/ClickHouse/ClickHouse/pull/51990) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that functional tests cleanup their tables [#51991](https://github.com/ClickHouse/ClickHouse/pull/51991) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_extreme_deduplication [#51992](https://github.com/ClickHouse/ClickHouse/pull/51992) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Cleanup SymbolIndex after reload got removed [#51993](https://github.com/ClickHouse/ClickHouse/pull/51993) ([Azat Khuzhin](https://github.com/azat)). +* Update CompletedPipelineExecutor exception log name [#52028](https://github.com/ClickHouse/ClickHouse/pull/52028) ([xiao](https://github.com/nicelulu)). +* Fix `00502_custom_partitioning_replicated_zookeeper_long` [#52032](https://github.com/ClickHouse/ClickHouse/pull/52032) ([Antonio Andelic](https://github.com/antonio2368)). +* Prohibit send_metadata for s3_plain disks [#52038](https://github.com/ClickHouse/ClickHouse/pull/52038) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.4.6.25-stable [#52061](https://github.com/ClickHouse/ClickHouse/pull/52061) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Preparations for Trivial Support For Resharding (part1) [#52068](https://github.com/ClickHouse/ClickHouse/pull/52068) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.3.8.21-lts [#52077](https://github.com/ClickHouse/ClickHouse/pull/52077) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix flakiness of test_keeper_s3_snapshot flakiness [#52083](https://github.com/ClickHouse/ClickHouse/pull/52083) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_extreme_deduplication flakiness [#52085](https://github.com/ClickHouse/ClickHouse/pull/52085) ([Azat Khuzhin](https://github.com/azat)). +* Small docs update for toYearWeek() function [#52090](https://github.com/ClickHouse/ClickHouse/pull/52090) ([Andrey Zvonov](https://github.com/zvonand)). +* Small docs update for DateTime, DateTime64 [#52094](https://github.com/ClickHouse/ClickHouse/pull/52094) ([Andrey Zvonov](https://github.com/zvonand)). +* Add missing --force for docker network prune (otherwise it is noop on CI) [#52095](https://github.com/ClickHouse/ClickHouse/pull/52095) ([Azat Khuzhin](https://github.com/azat)). +* tests: drop existing view in test_materialized_mysql_database [#52103](https://github.com/ClickHouse/ClickHouse/pull/52103) ([Azat Khuzhin](https://github.com/azat)). +* Update README.md [#52115](https://github.com/ClickHouse/ClickHouse/pull/52115) ([Tyler Hannan](https://github.com/tylerhannan)). +* Print Zxid in keeper stat command in hex (so as ZooKeeper) [#52122](https://github.com/ClickHouse/ClickHouse/pull/52122) ([Azat Khuzhin](https://github.com/azat)). +* Skip protection from double decompression if inode from maps cannot be obtained [#52138](https://github.com/ClickHouse/ClickHouse/pull/52138) ([Azat Khuzhin](https://github.com/azat)). +* There is no point in detecting flaky tests [#52142](https://github.com/ClickHouse/ClickHouse/pull/52142) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove default argument value [#52143](https://github.com/ClickHouse/ClickHouse/pull/52143) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the "kill_mutation" test [#52144](https://github.com/ClickHouse/ClickHouse/pull/52144) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ORDER BY tuple of WINDOW functions (and slightly more changes) [#52146](https://github.com/ClickHouse/ClickHouse/pull/52146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible EADDRINUSE ("Address already in use") in integration tests [#52148](https://github.com/ClickHouse/ClickHouse/pull/52148) ([Azat Khuzhin](https://github.com/azat)). +* Fix test 02497_storage_file_reader_selection [#52154](https://github.com/ClickHouse/ClickHouse/pull/52154) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix unexpected AST Set [#52158](https://github.com/ClickHouse/ClickHouse/pull/52158) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix crash in comparison functions due to incorrect query analysis [#52172](https://github.com/ClickHouse/ClickHouse/pull/52172) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix slow test `02317_distinct_in_order_optimization` [#52173](https://github.com/ClickHouse/ClickHouse/pull/52173) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add comments for https://github.com/ClickHouse/ClickHouse/pull/52112 [#52175](https://github.com/ClickHouse/ClickHouse/pull/52175) ([李扬](https://github.com/taiyang-li)). +* Randomize timezone in tests across non-deterministic around 1970 and default [#52184](https://github.com/ClickHouse/ClickHouse/pull/52184) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_multiple_disks/test.py::test_start_stop_moves` [#52189](https://github.com/ClickHouse/ClickHouse/pull/52189) ([Antonio Andelic](https://github.com/antonio2368)). +* CMake: Simplify job limiting [#52196](https://github.com/ClickHouse/ClickHouse/pull/52196) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) [#52198](https://github.com/ClickHouse/ClickHouse/pull/52198) ([Azat Khuzhin](https://github.com/azat)). +* Fix `Integration tests flaky check (asan)` [#52201](https://github.com/ClickHouse/ClickHouse/pull/52201) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky test test_lost_part [#52202](https://github.com/ClickHouse/ClickHouse/pull/52202) ([alesapin](https://github.com/alesapin)). +* MaterializedMySQL: Replace to_string by magic_enum::enum_name [#52204](https://github.com/ClickHouse/ClickHouse/pull/52204) ([Val Doroshchuk](https://github.com/valbok)). +* MaterializedMySQL: Add tests to parse db and table names from DDL [#52208](https://github.com/ClickHouse/ClickHouse/pull/52208) ([Val Doroshchuk](https://github.com/valbok)). +* Revert "Fixed several issues found by OSS-Fuzz" [#52216](https://github.com/ClickHouse/ClickHouse/pull/52216) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use one copy replication more agressively [#52218](https://github.com/ClickHouse/ClickHouse/pull/52218) ([alesapin](https://github.com/alesapin)). +* Fix flaky test `01076_parallel_alter_replicated_zookeeper` [#52221](https://github.com/ClickHouse/ClickHouse/pull/52221) ([alesapin](https://github.com/alesapin)). +* Fix 01889_key_condition_function_chains for analyzer. [#52223](https://github.com/ClickHouse/ClickHouse/pull/52223) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Inhibit settings randomization in the test `json_ghdata` [#52226](https://github.com/ClickHouse/ClickHouse/pull/52226) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Slightly better diagnostics in a test [#52227](https://github.com/ClickHouse/ClickHouse/pull/52227) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable no-upgrade-check for 02273_full_sort_join [#52235](https://github.com/ClickHouse/ClickHouse/pull/52235) ([vdimir](https://github.com/vdimir)). +* Fix network manager for integration tests [#52237](https://github.com/ClickHouse/ClickHouse/pull/52237) ([Azat Khuzhin](https://github.com/azat)). +* List replication queue only for current test database [#52238](https://github.com/ClickHouse/ClickHouse/pull/52238) ([Alexander Gololobov](https://github.com/davenger)). +* Attempt to fix assert in tsan with fibers [#52241](https://github.com/ClickHouse/ClickHouse/pull/52241) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix undefined behaviour in fuzzer [#52256](https://github.com/ClickHouse/ClickHouse/pull/52256) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow-up to [#51959](https://github.com/ClickHouse/ClickHouse/issues/51959) [#52261](https://github.com/ClickHouse/ClickHouse/pull/52261) ([Alexander Tokmakov](https://github.com/tavplubix)). +* More fair queue for `drop table sync` [#52276](https://github.com/ClickHouse/ClickHouse/pull/52276) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `02497_trace_events_stress_long` [#52279](https://github.com/ClickHouse/ClickHouse/pull/52279) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix test `01111_create_drop_replicated_db_stress` [#52283](https://github.com/ClickHouse/ClickHouse/pull/52283) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ugly code [#52284](https://github.com/ClickHouse/ClickHouse/pull/52284) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add missing replica syncs in test_backup_restore_on_cluster [#52306](https://github.com/ClickHouse/ClickHouse/pull/52306) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix test_replicated_database 'node doesn't exist' flakiness [#52307](https://github.com/ClickHouse/ClickHouse/pull/52307) ([Michael Kolupaev](https://github.com/al13n321)). +* Minor: Update description of events "QueryCacheHits/Misses" [#52309](https://github.com/ClickHouse/ClickHouse/pull/52309) ([Robert Schulze](https://github.com/rschu1ze)). +* Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE [#52312](https://github.com/ClickHouse/ClickHouse/pull/52312) ([Robert Schulze](https://github.com/rschu1ze)). +* Reduce dependencies for skim by avoid using default features [#52316](https://github.com/ClickHouse/ClickHouse/pull/52316) ([Azat Khuzhin](https://github.com/azat)). +* Fix 02725_memory-for-merges [#52317](https://github.com/ClickHouse/ClickHouse/pull/52317) ([alesapin](https://github.com/alesapin)). +* Skip unsupported disks in Keeper [#52321](https://github.com/ClickHouse/ClickHouse/pull/52321) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" [#52322](https://github.com/ClickHouse/ClickHouse/pull/52322) ([Kruglov Pavel](https://github.com/Avogar)). +* Resubmit [#51716](https://github.com/ClickHouse/ClickHouse/issues/51716) [#52323](https://github.com/ClickHouse/ClickHouse/pull/52323) ([Kruglov Pavel](https://github.com/Avogar)). +* Add logging about all found workflows for merge_pr.py [#52324](https://github.com/ClickHouse/ClickHouse/pull/52324) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Minor: Less awkward IAST::FormatSettings [#52332](https://github.com/ClickHouse/ClickHouse/pull/52332) ([Robert Schulze](https://github.com/rschu1ze)). +* Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness [#52338](https://github.com/ClickHouse/ClickHouse/pull/52338) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix capabilities installed via systemd service (fixes netlink/IO priorities) [#52357](https://github.com/ClickHouse/ClickHouse/pull/52357) ([Azat Khuzhin](https://github.com/azat)). +* Update 01606_git_import.sh [#52360](https://github.com/ClickHouse/ClickHouse/pull/52360) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update ci-slack-bot.py [#52372](https://github.com/ClickHouse/ClickHouse/pull/52372) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `test_keeper_session` [#52373](https://github.com/ClickHouse/ClickHouse/pull/52373) ([Antonio Andelic](https://github.com/antonio2368)). +* Update ci-slack-bot.py [#52374](https://github.com/ClickHouse/ClickHouse/pull/52374) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable analyzer setting in backward_compatibility integration tests. [#52375](https://github.com/ClickHouse/ClickHouse/pull/52375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* New metric - Filesystem cache size limit [#52378](https://github.com/ClickHouse/ClickHouse/pull/52378) ([Krzysztof Góralski](https://github.com/kgoralski)). +* Fix `test_replicated_merge_tree_encrypted_disk ` [#52379](https://github.com/ClickHouse/ClickHouse/pull/52379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `02122_parallel_formatting_XML ` [#52380](https://github.com/ClickHouse/ClickHouse/pull/52380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Follow up to [#49698](https://github.com/ClickHouse/ClickHouse/issues/49698) [#52381](https://github.com/ClickHouse/ClickHouse/pull/52381) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Less replication errors [#52382](https://github.com/ClickHouse/ClickHouse/pull/52382) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rename TaskStatsInfoGetter into NetlinkMetricsProvider [#52392](https://github.com/ClickHouse/ClickHouse/pull/52392) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_keeper_force_recovery` [#52408](https://github.com/ClickHouse/ClickHouse/pull/52408) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky gtest_lru_file_cache.cpp [#52418](https://github.com/ClickHouse/ClickHouse/pull/52418) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix: remove redundant distinct with views [#52438](https://github.com/ClickHouse/ClickHouse/pull/52438) ([Igor Nikonov](https://github.com/devcrafter)). +* Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt [#52464](https://github.com/ClickHouse/ClickHouse/pull/52464) ([vdimir](https://github.com/vdimir)). +* do not throw exception in OptimizedRegularExpressionImpl::analyze [#52467](https://github.com/ClickHouse/ClickHouse/pull/52467) ([Han Fei](https://github.com/hanfei1991)). +* Remove skip_startup_tables from IDatabase::loadStoredObjects() [#52491](https://github.com/ClickHouse/ClickHouse/pull/52491) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_insert_same_partition_and_merge by increasing wait time [#52497](https://github.com/ClickHouse/ClickHouse/pull/52497) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Try to fix asan wanring in HashJoin [#52499](https://github.com/ClickHouse/ClickHouse/pull/52499) ([Igor Nikonov](https://github.com/devcrafter)). +* Replace with three way comparison [#52509](https://github.com/ClickHouse/ClickHouse/pull/52509) ([flynn](https://github.com/ucasfl)). +* Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop [#52514](https://github.com/ClickHouse/ClickHouse/pull/52514) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_throttling` [#52515](https://github.com/ClickHouse/ClickHouse/pull/52515) ([Antonio Andelic](https://github.com/antonio2368)). +* Improve logging macros [#52519](https://github.com/ClickHouse/ClickHouse/pull/52519) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `toDecimalString` function [#52520](https://github.com/ClickHouse/ClickHouse/pull/52520) ([Andrey Zvonov](https://github.com/zvonand)). +* Remove unused code [#52527](https://github.com/ClickHouse/ClickHouse/pull/52527) ([Raúl Marín](https://github.com/Algunenano)). +* Cancel execution in PipelineExecutor in case of exception in graph->updateNode [#52533](https://github.com/ClickHouse/ClickHouse/pull/52533) ([Kruglov Pavel](https://github.com/Avogar)). +* Make 01951_distributed_push_down_limit analyzer agnostic [#52534](https://github.com/ClickHouse/ClickHouse/pull/52534) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix disallow_concurrency test for backup and restore [#52536](https://github.com/ClickHouse/ClickHouse/pull/52536) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update 02136_scalar_subquery_metrics.sql [#52537](https://github.com/ClickHouse/ClickHouse/pull/52537) ([Alexander Tokmakov](https://github.com/tavplubix)). +* tests: fix 01035_avg_weighted_long flakiness [#52556](https://github.com/ClickHouse/ClickHouse/pull/52556) ([Azat Khuzhin](https://github.com/azat)). +* tests: increase throttling for 01923_network_receive_time_metric_insert [#52557](https://github.com/ClickHouse/ClickHouse/pull/52557) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 00719_parallel_ddl_table flakiness in debug builds [#52558](https://github.com/ClickHouse/ClickHouse/pull/52558) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 01821_join_table_race_long flakiness [#52559](https://github.com/ClickHouse/ClickHouse/pull/52559) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky `00995_exception_while_insert` [#52568](https://github.com/ClickHouse/ClickHouse/pull/52568) ([Antonio Andelic](https://github.com/antonio2368)). +* MaterializedMySQL: Fix typos in tests [#52575](https://github.com/ClickHouse/ClickHouse/pull/52575) ([Val Doroshchuk](https://github.com/valbok)). +* Fix `02497_trace_events_stress_long` again [#52587](https://github.com/ClickHouse/ClickHouse/pull/52587) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Remove `mmap/mremap/munmap` from Allocator.h" [#52589](https://github.com/ClickHouse/ClickHouse/pull/52589) ([Nikita Taranov](https://github.com/nickitat)). +* Remove peak memory usage from the final message in the client [#52598](https://github.com/ClickHouse/ClickHouse/pull/52598) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* GinIndexStore: fix a bug when files are finalizated after first write, [#52602](https://github.com/ClickHouse/ClickHouse/pull/52602) ([Sema Checherinda](https://github.com/CheSema)). +* Fix deadlocks in StorageTableFunctionProxy [#52626](https://github.com/ClickHouse/ClickHouse/pull/52626) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build with clang-15 [#52627](https://github.com/ClickHouse/ClickHouse/pull/52627) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix style [#52647](https://github.com/ClickHouse/ClickHouse/pull/52647) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix logging level of a noisy message [#52648](https://github.com/ClickHouse/ClickHouse/pull/52648) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Revert "Added field `refcount` to `system.remote_data_paths` table" [#52657](https://github.com/ClickHouse/ClickHouse/pull/52657) ([Alexander Tokmakov](https://github.com/tavplubix)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8b535e3d897..1eabc65a10f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.7.1.2470-stable 2023-07-27 v23.6.2.18-stable 2023-07-09 v23.6.1.1524-stable 2023-06-30 v23.5.4.25-stable 2023-06-29 From ce38d3c5ea45507696430e9c7f39f9ab7b9de394 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:11:08 +0200 Subject: [PATCH 800/871] address comment --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 53481ab06a0..dbb4f7f0d8e 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,11 +10,17 @@ #include #include +#include "Common/Exception.h" #include namespace DB { +namespace ErrorCodes +{ + extern const int ABORTED; +} + namespace { @@ -271,8 +277,7 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); - return; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f5dfb70f5c2f4b94a54e9fdb97737a70b28362ad Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:12:20 +0200 Subject: [PATCH 801/871] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index dbb4f7f0d8e..80e94b2fd39 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,7 +10,7 @@ #include #include -#include "Common/Exception.h" +#include #include namespace DB From b5160c8072b9660e7c62b8305f56eda26313ecb0 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 14:16:19 +0000 Subject: [PATCH 802/871] Fix bugs and better test for SYSTEM STOP LISTEN --- programs/server/Server.cpp | 25 ++-- src/Server/ServerType.cpp | 20 +-- src/Server/ServerType.h | 1 - .../configs/cluster.xml | 4 +- .../configs/protocols.xml | 23 ++++ .../test_system_start_stop_listen/test.py | 128 ++++++++++++++++-- 6 files changed, 157 insertions(+), 44 deletions(-) create mode 100644 tests/integration/test_system_start_stop_listen/configs/protocols.xml diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index dce52ecdb12..adec13a5199 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2031,27 +2031,26 @@ void Server::createServers( for (const auto & protocol : protocols) { - if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + std::string prefix = "protocols." + protocol + "."; + std::string port_name = prefix + "port"; + std::string description {" protocol"}; + if (config.has(prefix + "description")) + description = config.getString(prefix + "description"); + + if (!config.has(prefix + "port")) + continue; + + if (!server_type.shouldStart(ServerType::Type::CUSTOM, port_name)) continue; std::vector hosts; - if (config.has("protocols." + protocol + ".host")) - hosts.push_back(config.getString("protocols." + protocol + ".host")); + if (config.has(prefix + "host")) + hosts.push_back(config.getString(prefix + "host")); else hosts = listen_hosts; for (const auto & host : hosts) { - std::string conf_name = "protocols." + protocol; - std::string prefix = conf_name + "."; - - if (!config.has(prefix + "port")) - continue; - - std::string description {" protocol"}; - if (config.has(prefix + "description")) - description = config.getString(prefix + "description"); - std::string port_name = prefix + "port"; bool is_secure = false; auto stack = buildProtocolStackFromConfig(config, protocol, http_params, async_metrics, is_secure); diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index c6916ee39d9..4952cd1bd24 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -40,7 +40,7 @@ const char * ServerType::serverTypeToString(ServerType::Type type) return type_name.data(); } -bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) const +bool ServerType::shouldStart(Type server_type, const std::string & server_custom_name) const { if (type == Type::QUERIES_ALL) return true; @@ -77,13 +77,15 @@ bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) } } - return type == server_type && custom_name == custom_name_; + if (type == Type::CUSTOM) + return server_type == type && server_custom_name == "protocols." + custom_name + ".port"; + + return server_type == type; } bool ServerType::shouldStop(const std::string & port_name) const { Type port_type; - std::string port_custom_name; if (port_name == "http_port") port_type = Type::HTTP; @@ -119,20 +121,12 @@ bool ServerType::shouldStop(const std::string & port_name) const port_type = Type::INTERSERVER_HTTPS; else if (port_name.starts_with("protocols.") && port_name.ends_with(".port")) - { - constexpr size_t protocols_size = std::string_view("protocols.").size(); - constexpr size_t port_size = std::string_view("protocols.").size(); - port_type = Type::CUSTOM; - port_custom_name = port_name.substr(protocols_size, port_name.size() - port_size); - } - else - port_type = Type::UNKNOWN; - if (port_type == Type::UNKNOWN) + else return false; - return shouldStart(type, port_custom_name); + return shouldStart(port_type, port_name); } } diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 345d1a10119..8fb81622ab3 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -10,7 +10,6 @@ public: enum Type { - UNKNOWN, TCP, TCP_WITH_PROXY, TCP_SECURE, diff --git a/tests/integration/test_system_start_stop_listen/configs/cluster.xml b/tests/integration/test_system_start_stop_listen/configs/cluster.xml index 93d8f890f40..34b6c32c6d0 100644 --- a/tests/integration/test_system_start_stop_listen/configs/cluster.xml +++ b/tests/integration/test_system_start_stop_listen/configs/cluster.xml @@ -3,11 +3,11 @@ - node1 + main_node 9000 - node2 + backup_node 9000 diff --git a/tests/integration/test_system_start_stop_listen/configs/protocols.xml b/tests/integration/test_system_start_stop_listen/configs/protocols.xml new file mode 100644 index 00000000000..1d8608bcaca --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/configs/protocols.xml @@ -0,0 +1,23 @@ + + 0.0.0.0 + + + 9000 + 8123 + 9004 + + + + + tcp + 0.0.0.0 + 9001 + native protocol (tcp) + + + http + 8124 + http protocol + + + diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index ec1a000c599..0db313368fd 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -2,20 +2,16 @@ import pytest -import time from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry -import random -import string -import json +from helpers.client import Client +import requests cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", main_configs=["configs/cluster.xml"], with_zookeeper=True +main_node = cluster.add_instance( + "main_node", main_configs=["configs/cluster.xml", "configs/protocols.xml"], with_zookeeper=True ) -node2 = cluster.add_instance( - "node2", main_configs=["configs/cluster.xml"], with_zookeeper=True +backup_node = cluster.add_instance( + "backup_node", main_configs=["configs/cluster.xml"], with_zookeeper=True ) @@ -30,11 +26,113 @@ def started_cluster(): cluster.shutdown() -def test_system_start_stop_listen_queries(started_cluster): - node1.query("SYSTEM STOP LISTEN QUERIES ALL") +def http_works(port=8123): + try: + response = requests.post(f"http://{main_node.ip_address}:{port}/ping") + if response.status_code == 400: + return True + except: + pass - assert "Connection refused" in node1.query_and_get_error("SELECT 1", timeout=3) + return False - node2.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") +def assert_everything_works(): + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + main_node.query(QUERY) + main_node.query(MYSQL_QUERY) + custom_client.query(QUERY) + assert http_works() + assert http_works(8124) - node1.query("SELECT 1") +QUERY = "SELECT 1" +MYSQL_QUERY = "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100)" + +def test_default_protocols(started_cluster): + # TCP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN TCP") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + backup_node.query("SYSTEM START LISTEN ON CLUSTER default TCP") + + # HTTP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN HTTP") + assert http_works() == False + main_node.query("SYSTEM START LISTEN HTTP") + + # MySQL + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN MYSQL") + assert "Connections to mysql failed" in main_node.query_and_get_error(MYSQL_QUERY) + main_node.query("SYSTEM START LISTEN MYSQL") + + assert_everything_works() + +def test_custom_protocols(started_cluster): + # TCP + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN CUSTOM 'tcp'") + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + main_node.query("SYSTEM START LISTEN CUSTOM 'tcp'") + + # HTTP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN CUSTOM 'http'") + assert http_works(8124) == False + main_node.query("SYSTEM START LISTEN CUSTOM 'http'") + + assert_everything_works() + +def test_all_protocols(started_cluster): + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + assert_everything_works() + + # STOP LISTEN QUERIES ALL + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() == False + assert http_works(8124) == False + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + # STOP LISTEN QUERIES DEFAULT + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES DEFAULT") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES DEFAULT") + + # STOP LISTEN QUERIES CUSTOM + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES CUSTOM") + main_node.query(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() + assert http_works(8124) == False + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES CUSTOM") + + # Disable all protocols, check first START LISTEN QUERIES DEFAULT then START LISTEN QUERIES CUSTOM + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES DEFAULT") + main_node.query(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() + assert http_works(8124) == False + + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES CUSTOM") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) + + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + assert_everything_works() From ee9bad7a3140cc3164cbf36ef45486068b4be0e7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 14:18:46 +0000 Subject: [PATCH 803/871] Fix style --- tests/integration/test_system_start_stop_listen/test.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index 0db313368fd..1925685af03 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -8,7 +8,9 @@ import requests cluster = ClickHouseCluster(__file__) main_node = cluster.add_instance( - "main_node", main_configs=["configs/cluster.xml", "configs/protocols.xml"], with_zookeeper=True + "main_node", + main_configs=["configs/cluster.xml", "configs/protocols.xml"], + with_zookeeper=True, ) backup_node = cluster.add_instance( "backup_node", main_configs=["configs/cluster.xml"], with_zookeeper=True @@ -36,6 +38,7 @@ def http_works(port=8123): return False + def assert_everything_works(): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) main_node.query(QUERY) @@ -44,9 +47,11 @@ def assert_everything_works(): assert http_works() assert http_works(8124) + QUERY = "SELECT 1" MYSQL_QUERY = "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100)" + def test_default_protocols(started_cluster): # TCP assert_everything_works() @@ -68,6 +73,7 @@ def test_default_protocols(started_cluster): assert_everything_works() + def test_custom_protocols(started_cluster): # TCP custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) @@ -84,6 +90,7 @@ def test_custom_protocols(started_cluster): assert_everything_works() + def test_all_protocols(started_cluster): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) assert_everything_works() From 0d44d527ef590a5471ea577c132edb42f0c99c70 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:27:04 +0200 Subject: [PATCH 804/871] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 80e94b2fd39..25b93160d27 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -241,8 +241,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); - return {}; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f6ca013c536d76ca6c1403db5f84d792f6bd8864 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:00 +0200 Subject: [PATCH 805/871] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 25b93160d27..22dabc43a8c 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -11,7 +11,6 @@ #include #include -#include namespace DB { From 9488567bf6be7e2b751917a179222478fcb46f5e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:08 +0200 Subject: [PATCH 806/871] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22dabc43a8c..548b61ce422 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -240,7 +240,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we add this part."); } chassert(current_writer != nullptr); From 5611b2fff484d74c70c8ad6b62ba8d66c0b63589 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 27 Jul 2023 15:45:55 +0200 Subject: [PATCH 807/871] Add a note about not working _table filter for Merge with analyzer Signed-off-by: Azat Khuzhin --- src/Storages/StorageMerge.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b0ed242d14d..272f35303bd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -795,6 +795,10 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( bool filter_by_database_virtual_column /* = false */, bool filter_by_table_virtual_column /* = false */) const { + /// FIXME: filtering does not work with allow_experimental_analyzer due to + /// different column names there (it has "table_name._table" not just + /// "_table") + assert(!filter_by_database_virtual_column || !filter_by_table_virtual_column || query); const Settings & settings = query_context->getSettingsRef(); From 68aed0d16e331a6ba6b592243f10ce2a816152db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 26 Jul 2023 20:25:48 +0200 Subject: [PATCH 808/871] RFC: Fix filtering by virtual columns with OR expression Virtual columns did not supports queries with OR, for example query like this (here `m` is the `Merge` table, see the test): select key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1'); Will always leads to: Cannot find column `value` in source stream, there are only columns ... The reason for this is that it actually executes the following queries: SELECT key, value FROM default.d1 WHERE ((value = 10) AND ('v1' = 'v1')) OR ((value = 20) AND ('v1' = 'v1')); SELECT key FROM default.d2 WHERE 0; And this kind of filtering is used not only for `Merge` table but also: - `_table` for `Merge` (already mentioned) - `_file` for `File` - `_idx` for `S3` - and as well as filtering `system.*` tables by `database`/`table`/... Signed-off-by: Azat Khuzhin --- src/Storages/VirtualColumnUtils.cpp | 36 +++++++++++++----- .../02840_merge__table_or_filter.reference | 38 +++++++++++++++++++ .../02840_merge__table_or_filter.sql.j2 | 34 +++++++++++++++++ 3 files changed, 99 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.reference create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 907fc0cd22c..79be1f98a0f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -63,14 +64,31 @@ bool isValidFunction(const ASTPtr & expression, const std::function & is_constant, ASTs & result) { const auto * function = expression->as(); - if (function && (function->name == "and" || function->name == "indexHint")) + + if (function) { - bool ret = true; - for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, result); - return ret; + if (function->name == "and" || function->name == "indexHint") + { + bool ret = true; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, result); + return ret; + } + else if (function->name == "or") + { + bool ret = true; + ASTs or_args; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, or_args); + /// We can keep condition only if it still OR condition (i.e. we + /// have dependent conditions for columns at both sides) + if (or_args.size() == 2) + result.push_back(makeASTForLogicalOr(std::move(or_args))); + return ret; + } } - else if (isValidFunction(expression, is_constant)) + + if (isValidFunction(expression, is_constant)) { result.push_back(expression->clone()); return true; @@ -80,13 +98,13 @@ bool extractFunctions(const ASTPtr & expression, const std::function Date: Thu, 27 Jul 2023 09:49:34 +0000 Subject: [PATCH 809/871] Add query cache metrics to system.asynchronous_metrics Cf. https://github.com/ClickHouse/ClickHouse/pull/52384#issuecomment-1653241216 --- docs/en/operations/query-cache.md | 11 ++++++----- .../operations/system-tables/asynchronous_metrics.md | 12 ++++++++++++ docs/en/operations/system-tables/events.md | 2 ++ docs/en/operations/system-tables/metrics.md | 2 +- src/Interpreters/Cache/QueryCache.cpp | 10 ++++++++++ src/Interpreters/Cache/QueryCache.h | 5 ++++- src/Interpreters/ServerAsynchronousMetrics.cpp | 6 ++++++ 7 files changed, 41 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 547105c65cc..d0b785d8fda 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -61,11 +61,12 @@ use_query_cache = true`) but one should keep in mind that all `SELECT` queries i may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table -`system.query_cache`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table -[system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = -true". Other queries do not affect the cache miss counter. Field `query_log_usage` in system table -[system.query_log](system-tables/query_log.md) shows for each ran query whether the query result was written into or read from the query -cache. +`system.query_cache`. The number of query cache hits and misses since database start are shown as events "QueryCacheHits" and +"QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run +with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_log_usage` in system table +[system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or read from the +query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table +[system.asynchronous_metrics](system-tables/asynchronous_metrics.md) show how many entries / bytes the query cache currently contains. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be changed (see below) but doing so is not recommended for security reasons. diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index f357341da67..e46b495239c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -32,6 +32,10 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 └─────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + + ## Metric descriptions @@ -483,6 +487,14 @@ The value is similar to `OSUserTime` but divided to the number of CPU cores to b Number of threads in the server of the PostgreSQL compatibility protocol. +### QueryCacheBytes + +Total size of the query cache cache in bytes. + +### QueryCacheEntries + +Total number of entries in the query cache. + ### ReplicasMaxAbsoluteDelay Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data. diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index ba5602ee292..7846fe4be5d 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -11,6 +11,8 @@ Columns: - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. +You can find all supported events in source file [src/Common/ProfileEvents.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ProfileEvents.cpp). + **Example** ``` sql diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..b1dcea5500f 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -11,7 +11,7 @@ Columns: - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. -The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. +You can find all supported metrics in source file [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp). **Example** diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index a6c509e8bb1..5982a5ade50 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -496,6 +496,16 @@ void QueryCache::reset() cache_size_in_bytes = 0; } +size_t QueryCache::weight() const +{ + return cache.weight(); +} + +size_t QueryCache::count() const +{ + return cache.count(); +} + size_t QueryCache::recordQueryRun(const Key & key) { std::lock_guard lock(mutex); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c2de8ca22dd..eaa54c503fa 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -186,6 +186,9 @@ public: void reset(); + size_t weight() const; + size_t count() const; + /// Record new execution of query represented by key. Returns number of executions so far. size_t recordQueryRun(const Key & key); @@ -193,7 +196,7 @@ public: std::vector dump() const; private: - Cache cache; + Cache cache; /// has its own locking --> not protected by mutex mutable std::mutex mutex; TimesExecuted times_executed TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 0fbcfc9e6a1..68411e80755 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -92,6 +92,12 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values " The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."}; } + if (auto query_cache = getContext()->getQueryCache()) + { + new_values["QueryCacheBytes"] = { query_cache->weight(), "Total size of the query cache in bytes." }; + new_values["QueryCacheEntries"] = { query_cache->count(), "Total number of entries in the query cache." }; + } + { auto caches = FileCacheFactory::instance().getAll(); size_t total_bytes = 0; From dfc06d27143da106adc6d5fae3b5be089f1e2d64 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 27 Jul 2023 16:13:29 +0000 Subject: [PATCH 810/871] fix reading of unneded column in case of multistage prewhere --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 6 +++-- ...02833_multiprewhere_extra_column.reference | 2 ++ .../02833_multiprewhere_extra_column.sql | 25 +++++++++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02833_multiprewhere_extra_column.reference create mode 100644 tests/queries/0_stateless/02833_multiprewhere_extra_column.sql diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index d830ba37e71..48779aa0df7 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -340,8 +340,10 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (!columns_from_previous_steps.contains(name)) step_column_names.push_back(name); - injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + if (!step_column_names.empty()) + injectRequiredColumns( + data_part_info_for_reader, storage_snapshot, + with_subcolumns, step_column_names); /// More columns could have been added, filter them as well by the list of columns from previous steps. Names columns_to_read_in_step; diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference b/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference new file mode 100644 index 00000000000..45571c71477 --- /dev/null +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference @@ -0,0 +1,2 @@ +10496500 +4 diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql new file mode 100644 index 00000000000..a786de454ed --- /dev/null +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql @@ -0,0 +1,25 @@ +-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings + +drop table if exists t_multi_prewhere; +drop row policy if exists policy_02834 on t_multi_prewhere; + +create table t_multi_prewhere (a UInt64, b UInt64, c UInt8) +engine = MergeTree order by tuple() +settings min_bytes_for_wide_part = 0; + +create row policy policy_02834 on t_multi_prewhere using a > 2000 as permissive to all; +insert into t_multi_prewhere select number, number, number from numbers(10000); + +system drop mark cache; +select sum(b) from t_multi_prewhere prewhere a < 5000; + +system flush logs; + +select ProfileEvents['FileOpen'] from system.query_log +where + type = 'QueryFinish' + and current_database = currentDatabase() + and query ilike '%select sum(b) from t_multi_prewhere prewhere a < 5000%'; + +drop table if exists t_multi_prewhere; +drop row policy if exists policy_02834 on t_multi_prewhere; From 03f025ddd1352c264ceb716e731efc095f9cc0c3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 27 Jul 2023 16:38:02 +0000 Subject: [PATCH 811/871] Fix typo --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5dbe9e350bf..9fdc99bc760 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6530,7 +6530,7 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif array_join_column_expressions.push_back(std::move(array_join_column_node)); }; - // Support ARRAY JOIN COLUMNS(...). COLUMNS trasformer is resolved to list of columns. + // Support ARRAY JOIN COLUMNS(...). COLUMNS transformer is resolved to list of columns. if (auto * columns_list = array_join_expression->as()) { for (auto & array_join_subexpression : columns_list->getNodes()) From 380da315121078fc3e88a1e038e5aacd296853c2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 27 Jul 2023 19:03:44 +0200 Subject: [PATCH 812/871] Improvements to backup restore disallow_concurrency test --- .../test_disallow_concurrency.py | 102 +++++++++++++----- 1 file changed, 73 insertions(+), 29 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 a863a6e2047..af1b2656227 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,21 +133,31 @@ def test_concurrent_backups_on_same_node(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + + if not error: 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": + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -191,20 +201,31 @@ def test_concurrent_backups_on_different_nodes(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( + result, error = nodes[0].query_and_get_answer_with_error( f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + ) + + if not error: status = ( nodes[1] .query(f"SELECT status FROM system.backups WHERE id == '{id}'") .rstrip("\n") ) - if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[1], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -247,20 +268,32 @@ def test_concurrent_restores_on_same_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[0].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", @@ -303,20 +336,31 @@ def test_concurrent_restores_on_different_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[1].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[1].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", From a8a48af7cbd20334af531e9f71e7f6005a098db1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 27 Jul 2023 19:17:57 +0200 Subject: [PATCH 813/871] Fix 02417_opentelemetry_insert_on_distributed_table flakiness Looks like everything is OK with opentelemetry, and the reason of the flakiness is this: $ gg opentelemetry_start_trace_probability tests/**.xml tests/config/users.d/opentelemetry.xml: 0.1 So let's simply disable it. And also let's stop the distributed sends to increase the failure rate if there is some problem left. Signed-off-by: Azat Khuzhin --- .../02417_opentelemetry_insert_on_distributed_table.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index edc3d06e5bf..5a1e33a8459 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -20,7 +20,9 @@ function insert() -H "tracestate: $4" \ "${CLICKHOUSE_URL}" \ --data @- - ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry" + + # disable probabilistic tracing to avoid stealing the trace context + ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=0 -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry" } function check_span() @@ -69,6 +71,8 @@ DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.local_opentelemetry; CREATE TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry (key UInt64) Engine=Distributed('test_cluster_two_shards_localhost', ${CLICKHOUSE_DATABASE}, local_opentelemetry, key % 2); CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=MergeTree ORDER BY key; + +SYSTEM STOP DISTRIBUTED SENDS ${CLICKHOUSE_DATABASE}.dist_opentelemetry; " # From 2717be7c476d9d7c3be841bbefdcd8688a76e7fe Mon Sep 17 00:00:00 2001 From: Anton Kozlov Date: Thu, 27 Jul 2023 17:34:53 +0000 Subject: [PATCH 814/871] [minor][bugfix] fix connected_zk_index column of system.zookeeper_connection table --- src/Common/ZooKeeper/IKeeper.h | 2 -- src/Common/ZooKeeper/TestKeeper.h | 3 --- src/Common/ZooKeeper/ZooKeeper.cpp | 26 ++++++-------------------- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 14 ++++++++------ src/Common/ZooKeeper/ZooKeeperImpl.h | 8 ++++---- 5 files changed, 18 insertions(+), 35 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 5240acc2616..2b2a043d389 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -490,8 +490,6 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; - virtual Poco::Net::SocketAddress getConnectedAddress() const = 0; - /// If the method will throw an exception, callbacks won't be called. /// /// After the method is executed successfully, you must wait for callbacks diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 8615ed0fb77..69840cbeff6 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -40,7 +40,6 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } - Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void create( @@ -135,8 +134,6 @@ private: zkutil::ZooKeeperArgs args; - Poco::Net::SocketAddress connected_zk_address; - std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 7a8088c960b..0fe536b1a08 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -112,31 +112,17 @@ void ZooKeeper::init(ZooKeeperArgs args_) throw KeeperException("Cannot use any of provided ZooKeeper nodes", Coordination::Error::ZCONNECTIONLOSS); } - impl = std::make_unique(nodes, args, zk_log); + impl = std::make_unique(nodes, args, zk_log, [this](size_t node_idx, const Coordination::ZooKeeper::Node & node) + { + connected_zk_host = node.address.host().toString(); + connected_zk_port = node.address.port(); + connected_zk_index = node_idx; + }); if (args.chroot.empty()) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); - - Poco::Net::SocketAddress address = impl->getConnectedAddress(); - - connected_zk_host = address.host().toString(); - connected_zk_port = address.port(); - - connected_zk_index = 0; - - if (args.hosts.size() > 1) - { - for (size_t i = 0; i < args.hosts.size(); i++) - { - if (args.hosts[i] == address.toString()) - { - connected_zk_index = i; - break; - } - } - } } else if (args.implementation == "testkeeper") { diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 5e16a437be3..74b0b039ca9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -313,8 +313,8 @@ ZooKeeper::~ZooKeeper() ZooKeeper::ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_) - : args(args_) + std::shared_ptr zk_log_, std::optional && connected_callback_) + : args(args_), connected_callback(std::move(connected_callback_)) { log = &Poco::Logger::get("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); @@ -395,8 +395,9 @@ void ZooKeeper::connect( WriteBufferFromOwnString fail_reasons; for (size_t try_no = 0; try_no < num_tries; ++try_no) { - for (const auto & node : nodes) + for (size_t i = 0; i < nodes.size(); ++i) { + const auto & node = nodes[i]; try { /// Reset the state of previous attempt. @@ -443,9 +444,11 @@ void ZooKeeper::connect( e.addMessage("while receiving handshake from ZooKeeper"); throw; } - connected = true; - connected_zk_address = node.address; + + if (connected_callback.has_value()) + (*connected_callback)(i, node); + break; } @@ -462,7 +465,6 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - connected_zk_address = Poco::Net::SocketAddress(); message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 7e27608d0a1..3684b215144 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -107,6 +107,7 @@ public: }; using Nodes = std::vector; + using ConnectedCallback = std::function; /** Connection to nodes is performed in order. If you want, shuffle them manually. * Operation timeout couldn't be greater than session timeout. @@ -115,7 +116,8 @@ public: ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_); + std::shared_ptr zk_log_, + std::optional && connected_callback_ = {}); ~ZooKeeper() override; @@ -126,8 +128,6 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } - Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } - void executeGenericRequest( const ZooKeeperRequestPtr & request, ResponseCallback callback); @@ -213,9 +213,9 @@ public: private: ACLs default_acls; - Poco::Net::SocketAddress connected_zk_address; zkutil::ZooKeeperArgs args; + std::optional connected_callback = {}; /// Fault injection void maybeInjectSendFault(); From b95745d916330abb3306016d512b40d3d24616dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 17:52:44 +0000 Subject: [PATCH 815/871] fix: check positional options --- src/Client/ClientBase.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 496fc8fce0a..06dabf96c28 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,6 +2624,10 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } + /// Check positional options. + if (std::ranges::count_if(parsed.options, [](const auto & op){ return !op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--"); }) > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); + po::store(parsed, options); } From 9340f02d26ae7f170611ea9b19a11e720b41b765 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 18:33:07 +0000 Subject: [PATCH 816/871] Silence spell check --- .../aspell-ignore/en/aspell-dict.txt | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..80aeadd8738 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -211,7 +211,6 @@ Decrypted Deduplicate Deduplication DelayedInserts -delim DeliveryTag DeltaLake Denormalize @@ -699,6 +698,8 @@ PyCharm QEMU QTCreator Quantile +QueryCacheBytes +QueryCacheEntries QueryCacheHits QueryCacheMisses QueryPreempted @@ -761,9 +762,9 @@ RoaringBitmap RocksDB Rollup RowBinary +RowBinaryWithDefaults RowBinaryWithNames RowBinaryWithNamesAndTypes -RowBinaryWithDefaults Runtime SATA SELECTs @@ -776,7 +777,6 @@ SMALLINT SPNEGO SQEs SQLAlchemy -SquaredDistance SQLConsoleDetail SQLInsert SQLSTATE @@ -811,6 +811,7 @@ Smirnov'test Soundex SpanKind Spearman's +SquaredDistance StartTLS StartTime StartupSystemTables @@ -838,8 +839,6 @@ Subexpression Submodules Subqueries Substrings -substringIndex -substringIndexUTF SummingMergeTree SuperSet Superset @@ -1272,6 +1271,7 @@ cryptographic csv csvwithnames csvwithnamesandtypes +curdate currentDatabase currentProfiles currentRoles @@ -1331,6 +1331,7 @@ defaultProfiles defaultRoles defaultValueOfArgumentType defaultValueOfTypeName +delim deltaLake deltaSum deltaSumTimestamp @@ -1542,13 +1543,13 @@ hadoop halfMD halfday hardlinks +hasAll +hasAny +hasColumnInTable hasSubsequence hasSubsequenceCaseInsensitive hasSubsequenceCaseInsensitiveUTF hasSubsequenceUTF -hasAll -hasAny -hasColumnInTable hasSubstr hasToken hasTokenCaseInsensitive @@ -1590,10 +1591,10 @@ incrementing indexHint indexOf infi -initialQueryID -initializeAggregation initcap initcapUTF +initialQueryID +initializeAggregation injective innogames inodes @@ -2131,9 +2132,9 @@ routineley rowNumberInAllBlocks rowNumberInBlock rowbinary +rowbinarywithdefaults rowbinarywithnames rowbinarywithnamesandtypes -rowbinarywithdefaults rsync rsyslog runnable @@ -2185,8 +2186,8 @@ sleepEachRow snowflakeToDateTime socketcache soundex -sparkbar sparkBar +sparkbar sparsehash speedscope splitByChar @@ -2256,6 +2257,8 @@ subreddits subseconds subsequence substring +substringIndex +substringIndexUTF substringUTF substrings subtitiles @@ -2556,4 +2559,3 @@ znode znodes zookeeperSessionUptime zstd -curdate From 5942c80faed38febea5394526b5e5c670b03bd4d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 19:11:44 +0000 Subject: [PATCH 817/871] fix test: '--option' now is allowed after terminating '--' --- .../02096_bad_options_in_client_and_local.reference | 2 -- .../0_stateless/02096_bad_options_in_client_and_local.sh | 4 ---- 2 files changed, 6 deletions(-) diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference index c4c0901b9df..432299e9556 100644 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference @@ -8,5 +8,3 @@ OK OK OK OK -OK -OK diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh index d37155e8506..753d56fb424 100755 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh @@ -9,8 +9,6 @@ ${CLICKHOUSE_LOCAL} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_LOCAL} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_LOCAL} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_LOCAL} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_LOCAL} -f 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" @@ -22,8 +20,6 @@ ${CLICKHOUSE_CLIENT} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_CLIENT} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_CLIENT} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_CLIENT} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_CLIENT} --j 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" From eb74e658b7dac9cbfbd4027d2281ab4fc5c173a1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 19:12:06 +0000 Subject: [PATCH 818/871] Fix build --- src/Server/ServerType.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 8fb81622ab3..1fab492222a 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -33,7 +33,8 @@ public: static const char * serverTypeToString(Type type); - bool shouldStart(Type server_type, const std::string & custom_name_ = "") const; + /// Checks whether provided in the arguments type should be started or stopped based on current server type. + bool shouldStart(Type server_type, const std::string & server_custom_name = "") const; bool shouldStop(const std::string & port_name) const; Type type; From dba8b445bd37b2fb9fb4983e0a3f740649dcbb5b Mon Sep 17 00:00:00 2001 From: Jai Jhala Date: Thu, 27 Jul 2023 12:32:53 -0700 Subject: [PATCH 819/871] Update default output_format_arrow_compression.md Updates the default parameter of output_format_arrow_compression_method from "none" to "lz4_frame". --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index ee8e0d547b8..fb10ff7f61b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1164,7 +1164,7 @@ Enabled by default. Compression method used in output Arrow format. Supported codecs: `lz4_frame`, `zstd`, `none` (uncompressed) -Default value: `none`. +Default value: `lz4_frame`. ## ORC format settings {#orc-format-settings} From 4629ab1df1cf6e5de3577ff94f7bd600207f24b6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 27 Jul 2023 23:44:32 +0400 Subject: [PATCH 820/871] add test with broken pipe --- .../integration/helpers/s3_mocks/broken_s3.py | 80 +++++++++++++------ .../test_checking_s3_blobs_paranoid/test.py | 77 ++++++++++++++++-- 2 files changed, 126 insertions(+), 31 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 6e1572af262..206f960293f 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -37,9 +37,7 @@ class MockControl: ) assert response == "OK", response - def setup_action( - self, when, count=None, after=None, action="error_500", action_args=None - ): + def setup_action(self, when, count=None, after=None, action=None, action_args=None): url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: @@ -128,8 +126,14 @@ class MockControl: class _ServerRuntime: class SlowPut: def __init__( - self, probability_=None, timeout_=None, minimal_length_=None, count_=None + self, + lock, + probability_=None, + timeout_=None, + minimal_length_=None, + count_=None, ): + self.lock = lock self.probability = probability_ if probability_ is not None else 1 self.timeout = timeout_ if timeout_ is not None else 0.1 self.minimal_length = minimal_length_ if minimal_length_ is not None else 0 @@ -144,14 +148,15 @@ class _ServerRuntime: ) def get_timeout(self, content_length): - if content_length > self.minimal_length: - if self.count > 0: - if ( - _runtime.slow_put.probability == 1 - or random.random() <= _runtime.slow_put.probability - ): - self.count -= 1 - return _runtime.slow_put.timeout + with self.lock: + if content_length > self.minimal_length: + if self.count > 0: + if ( + _runtime.slow_put.probability == 1 + or random.random() <= _runtime.slow_put.probability + ): + self.count -= 1 + return _runtime.slow_put.timeout return None class Expected500ErrorAction: @@ -199,29 +204,48 @@ class _ServerRuntime: ) request_handler.connection.close() + class BrokenPipeAction: + def inject_error(self, request_handler): + # partial read + self.rfile.read(50) + + time.sleep(1) + request_handler.connection.setsockopt( + socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 1, 0) + ) + request_handler.connection.close() + class ConnectionRefusedAction(RedirectAction): pass class CountAfter: - def __init__(self, count_=None, after_=None, action_=None, action_args_=[]): + def __init__( + self, lock, count_=None, after_=None, action_=None, action_args_=[] + ): + self.lock = lock + self.count = count_ if count_ is not None else INF_COUNT self.after = after_ if after_ is not None else 0 self.action = action_ self.action_args = action_args_ + if self.action == "connection_refused": self.error_handler = _ServerRuntime.ConnectionRefusedAction() elif self.action == "connection_reset_by_peer": self.error_handler = _ServerRuntime.ConnectionResetByPeerAction( *self.action_args ) + elif self.action == "broken_pipe": + self.error_handler = _ServerRuntime.BrokenPipeAction() elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) else: self.error_handler = _ServerRuntime.Expected500ErrorAction() @staticmethod - def from_cgi_params(params): + def from_cgi_params(lock, params): return _ServerRuntime.CountAfter( + lock=lock, count_=_and_then(params.get("count", [None])[0], int), after_=_and_then(params.get("after", [None])[0], int), action_=params.get("action", [None])[0], @@ -232,13 +256,14 @@ class _ServerRuntime: return f"count:{self.count} after:{self.after} action:{self.action} action_args:{self.action_args}" def has_effect(self): - if self.after: - self.after -= 1 - if self.after == 0: - if self.count: - self.count -= 1 - return True - return False + with self.lock: + if self.after: + self.after -= 1 + if self.after == 0: + if self.count: + self.count -= 1 + return True + return False def inject_error(self, request_handler): self.error_handler.inject_error(request_handler) @@ -397,14 +422,16 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params( + _runtime.lock, params + ) self.log_message("set at_part_upload %s", _runtime.at_part_upload) return self._ok() if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params( - params + _runtime.lock, params ) self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() @@ -420,6 +447,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.slow_put = _ServerRuntime.SlowPut( + lock=_runtime.lock, minimal_length_=_and_then(params.get("minimal_length", [None])[0], int), probability_=_and_then(params.get("probability", [None])[0], float), timeout_=_and_then(params.get("timeout", [None])[0], float), @@ -436,7 +464,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.at_create_multi_part_upload = ( - _ServerRuntime.CountAfter.from_cgi_params(params) + _ServerRuntime.CountAfter.from_cgi_params(_runtime.lock, params) ) self.log_message( "set at_create_multi_part_upload %s", @@ -477,7 +505,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if upload_id is not None: if _runtime.at_part_upload is not None: self.log_message( - "put error_at_object_upload %s, %s, %s", + "put at_part_upload %s, %s, %s", _runtime.at_part_upload, upload_id, parts, @@ -492,7 +520,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if _runtime.at_object_upload is not None: if _runtime.at_object_upload.has_effect(): self.log_message( - "put error_at_object_upload %s, %s, %s", + "put error_at_object_upload %s, %s", _runtime.at_object_upload, parts, ) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 28b0c9beeaa..c40e2a31a8b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -41,11 +41,6 @@ def broken_s3(init_broken_s3): yield init_broken_s3 -@pytest.fixture(scope="module") -def init_connection_reset_by_peer(cluster): - yield start_s3_mock(cluster, "connection_reset_by_peer", "8084") - - def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] @@ -397,3 +392,75 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error + + +def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="broken_pipe", + ) + + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 7 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="broken_pipe", + ) + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" + in error + ), error From 7d8dc92ed0522e309760037720f6fd8fb3f2542d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 21:07:54 +0000 Subject: [PATCH 821/871] Automatic style fix --- .../test_disallow_concurrency.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 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 af1b2656227..5c3f06a9d9d 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 @@ -156,7 +156,9 @@ def test_concurrent_backups_on_same_node(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -202,7 +204,7 @@ def test_concurrent_backups_on_different_nodes(): assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] result, error = nodes[0].query_and_get_answer_with_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" ) if not error: @@ -224,7 +226,9 @@ def test_concurrent_backups_on_different_nodes(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -291,8 +295,9 @@ def test_concurrent_restores_on_same_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") - + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", @@ -359,7 +364,9 @@ def test_concurrent_restores_on_different_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", From 18c1fd6f08cc2be964ed15604c26a70d7d168561 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 27 Jul 2023 21:24:39 +0000 Subject: [PATCH 822/871] Refactor InDepthQueryTreeVisitorWithContext --- src/Analyzer/InDepthQueryTreeVisitor.h | 158 +++--------------- ...egateFunctionsArithmericOperationsPass.cpp | 8 +- src/Analyzer/Passes/ArrayExistsToHasPass.cpp | 2 +- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 2 +- .../Passes/ConvertOrLikeChainPass.cpp | 2 +- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 2 +- src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++-- src/Analyzer/Passes/CrossToInnerJoinPass.cpp | 2 +- .../Passes/FunctionToSubcolumnsPass.cpp | 2 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 +- .../Passes/GroupingFunctionsResolvePass.cpp | 2 +- src/Analyzer/Passes/IfChainToMultiIfPass.cpp | 2 +- .../Passes/IfTransformStringsToEnumPass.cpp | 2 +- .../Passes/LogicalExpressionOptimizerPass.cpp | 2 +- src/Analyzer/Passes/MultiIfToIfPass.cpp | 2 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- .../OptimizeGroupByFunctionKeysPass.cpp | 2 +- ...ptimizeRedundantFunctionsInOrderByPass.cpp | 2 +- .../RewriteAggregateFunctionWithIfPass.cpp | 2 +- .../Passes/ShardNumColumnToFunctionPass.cpp | 2 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 2 +- .../UniqInjectiveFunctionsEliminationPass.cpp | 2 +- src/Storages/buildQueryTreeForShard.cpp | 2 +- 23 files changed, 55 insertions(+), 185 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index be3a760d4e6..59ee57996c4 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -91,26 +91,25 @@ private: template using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; -/** Same as InDepthQueryTreeVisitor and additionally keeps track of current scope context. +/** Same as InDepthQueryTreeVisitor (but has a different interface) and additionally keeps track of current scope context. * This can be useful if your visitor has special logic that depends on current scope context. + * + * To specify behavior of the visitor you can implement following methods in derived class: + * 1. needChildVisit – This methods allows to skip subtree. + * 2. enterImpl – This method is called before children are processed. + * 3. leaveImpl – This method is called after children are processed. */ template class InDepthQueryTreeVisitorWithContext { public: - using VisitQueryTreeNodeType = std::conditional_t; + using VisitQueryTreeNodeType = QueryTreeNodePtr; explicit InDepthQueryTreeVisitorWithContext(ContextPtr context, size_t initial_subquery_depth = 0) : current_context(std::move(context)) , subquery_depth(initial_subquery_depth) {} - /// Return true if visitor should traverse tree top to bottom, false otherwise - bool shouldTraverseTopToBottom() const - { - return true; - } - /// Return true if visitor should visit child, false otherwise bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]]) { @@ -147,18 +146,16 @@ public: ++subquery_depth; - bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom(); - if (!traverse_top_to_bottom) - visitChildren(query_tree_node); + getDerived().enterImpl(query_tree_node); - getDerived().visitImpl(query_tree_node); - - if (traverse_top_to_bottom) - visitChildren(query_tree_node); + visitChildren(query_tree_node); getDerived().leaveImpl(query_tree_node); } + void enterImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) + {} + void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) {} private: @@ -172,85 +169,15 @@ private: return *static_cast(this); } - void visitChildren(VisitQueryTreeNodeType & expression) + bool shouldSkipSubtree( + VisitQueryTreeNodeType & parent, + VisitQueryTreeNodeType & child, + size_t subtree_index) { - for (auto & child : expression->getChildren()) - { - if (!child) - continue; + bool need_visit_child = getDerived().needChildVisit(parent, child); + if (!need_visit_child) + return true; - bool need_visit_child = getDerived().needChildVisit(expression, child); - - if (need_visit_child) - visit(child); - } - } - - ContextPtr current_context; - size_t subquery_depth = 0; -}; - -template -using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext; - -/** Visitor that use another visitor to visit node only if condition for visiting node is true. - * For example, your visitor need to visit only query tree nodes or union nodes. - * - * Condition interface: - * struct Condition - * { - * bool operator()(VisitQueryTreeNodeType & node) - * { - * return shouldNestedVisitorVisitNode(node); - * } - * } - */ -template -class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor, const_visitor> -{ -public: - using Base = InDepthQueryTreeVisitor, const_visitor>; - using VisitQueryTreeNodeType = typename Base::VisitQueryTreeNodeType; - - explicit InDepthQueryTreeConditionalVisitor(Visitor & visitor_, Condition & condition_) - : visitor(visitor_) - , condition(condition_) - { - } - - bool shouldTraverseTopToBottom() const - { - return visitor.shouldTraverseTopToBottom(); - } - - void visitImpl(VisitQueryTreeNodeType & query_tree_node) - { - if (condition(query_tree_node)) - visitor.visit(query_tree_node); - } - - Visitor & visitor; - Condition & condition; -}; - -template -using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; - -template -class QueryTreeVisitor -{ -public: - explicit QueryTreeVisitor(ContextPtr context_) - : current_context(std::move(context_)) - {} - - bool needApply(QueryTreeNodePtr & node) - { - return getImpl().needApply(node); - } - - bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) - { if (auto * table_function_node = parent->as()) { const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); @@ -259,58 +186,19 @@ public: return false; } - void visit(QueryTreeNodePtr & node) - { - auto current_scope_context_ptr = current_context; - SCOPE_EXIT( - current_context = std::move(current_scope_context_ptr); - ); - - if (auto * query_node = node->template as()) - current_context = query_node->getContext(); - else if (auto * union_node = node->template as()) - current_context = union_node->getContext(); - - if (!TOP_TO_BOTTOM) - visitChildren(node); - - if (needApply(node)) - getImpl().apply(node); - - if (TOP_TO_BOTTOM) - visitChildren(node); - } - - const ContextPtr & getContext() const - { - return current_context; - } - - const Settings & getSettings() const - { - return current_context->getSettingsRef(); - } -private: - - Impl & getImpl() - { - return *static_cast(this); - } - - void visitChildren(QueryTreeNodePtr & node) + void visitChildren(VisitQueryTreeNodeType & expression) { size_t index = 0; - for (auto & child : node->getChildren()) + for (auto & child : expression->getChildren()) { - if (child && !shouldSkipSubtree(node, index)) + if (child && !shouldSkipSubtree(expression, child, index)) visit(child); ++index; } } - static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; - ContextPtr current_context; + size_t subquery_depth = 0; }; } diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index 1476a66c892..3615a632374 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -51,13 +51,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - /// Traverse tree bottom to top - static bool shouldTraverseTopToBottom() - { - return false; - } - - void visitImpl(QueryTreeNodePtr & node) + void leaveImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions) return; diff --git a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp index c0f958588f1..a95bcea4fac 100644 --- a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp +++ b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp @@ -22,7 +22,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_array_exists_to_has) return; diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 15326ca1dc8..2c89ec9dc20 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().final) return; diff --git a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp index 7d7362fb742..1fada88a21c 100644 --- a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp +++ b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp @@ -50,7 +50,7 @@ public: && settings.max_hyperscan_regexp_total_length == 0; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "or") diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 4d32c96b845..724448ad742 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -688,7 +688,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); if (!query_node) diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 38f7d07d052..dc58747221e 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,17 +16,16 @@ namespace DB namespace { -class CountDistinctVisitor : public QueryTreeVisitor +class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext { public: - using QueryTreeVisitor::QueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; - static constexpr bool TOP_TO_BOTTOM = true; - - bool needApply(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return false; + return; auto * query_node = node->as(); @@ -34,43 +33,32 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return false; + return; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return false; + return; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return false; + return; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return false; + return; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return false; + return; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return false; - - return true; - } - - void apply(QueryTreeNodePtr & node) - { - auto * query_node = node->as(); - auto & projection_nodes = query_node->getProjection().getNodes(); - auto * function_node = projection_nodes[0]->as(); - - auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); + return; auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index d4877d23f28..b5ece1a4c49 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -193,7 +193,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!isEnabled()) return; diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 696483862e0..cd635f87e0e 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -29,7 +29,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { if (!getSettings().optimize_functions_to_subcolumns) return; diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 14082697955..2cb7afa4ad6 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -37,7 +37,7 @@ public: , names_to_collect(names_to_collect_) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_syntax_fuse_functions) return; diff --git a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp index 0cf5310a3ad..577bca8d1ae 100644 --- a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp +++ b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp @@ -46,7 +46,7 @@ public: { } - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "grouping") diff --git a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp index 1f97e012331..b0018d474d5 100644 --- a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp @@ -23,7 +23,7 @@ public: , multi_if_function_ptr(std::move(multi_if_function_ptr_)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_if_chain_to_multiif) return; diff --git a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp index 562aff4cf05..901867b8889 100644 --- a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp +++ b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp @@ -113,7 +113,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_if_transform_strings_to_enum) return; diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 13f8025f5ea..46056aeaf6f 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -19,7 +19,7 @@ public: : Base(std::move(context)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); diff --git a/src/Analyzer/Passes/MultiIfToIfPass.cpp b/src/Analyzer/Passes/MultiIfToIfPass.cpp index 4672351bcfb..85dd33af8bb 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.cpp +++ b/src/Analyzer/Passes/MultiIfToIfPass.cpp @@ -21,7 +21,7 @@ public: , if_function_ptr(std::move(if_function_ptr_)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_multiif_to_if) return; diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index d36be98751c..c85b863a203 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_normalize_count_variants) return; diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index 5ed52f1210b..2e3f207fdeb 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -26,7 +26,7 @@ public: return !child->as(); } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_group_by_function_keys) return; diff --git a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp index c6d312d0ecf..875d0c8b5fb 100644 --- a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp +++ b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp @@ -28,7 +28,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_redundant_functions_in_order_by) return; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index de264948d4c..38f2fbfa274 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_aggregate_function_with_if) return; diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp index b28816e8ff3..52c30b7b35d 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -24,7 +24,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { auto * column_node = node->as(); if (!column_node) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index d55af278152..cff9ba1111c 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_sum_if_to_count_if) return; diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index 5c4484457e8..179bd1c38e4 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -31,7 +31,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_injective_functions_inside_uniq) return; diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..9929b5bb39b 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -130,7 +130,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); auto * join_node = node->as(); From 6573ba537819ce03dd644ff02bdf7341bcc26d58 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 19:37:28 -0400 Subject: [PATCH 823/871] Temporary returning metadata_cache.xml into tests config --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9aaadbc74a5..50f2627d37c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -34,6 +34,7 @@ ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/merge_tree.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/lost_forever_check.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/metadata_cache.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/prometheus.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ From b3351bb547b8753b405d820925f8f4270be6132d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 03:36:23 +0000 Subject: [PATCH 824/871] partially fixed 01747_system_session_log_long test --- src/Core/PostgreSQLProtocol.h | 49 +++-- tests/config/users.d/session_log_test.xml | 2 +- .../01747_system_session_log_long.reference | 198 ++++++++++++------ .../01747_system_session_log_long.sh | 119 ++++++----- 4 files changed, 224 insertions(+), 144 deletions(-) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.reference (73%) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.sh (78%) diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 8c0654b559f..b0d7646a5f7 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -805,20 +805,9 @@ protected: const String & user_name, const String & password, Session & session, - Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - try - { - session.authenticate(user_name, password, address); - } - catch (const Exception &) - { - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), - true); - throw; - } + session.authenticate(user_name, password, address); } public: @@ -839,10 +828,10 @@ public: void authenticate( const String & user_name, Session & session, - Messaging::MessageTransport & mt, + [[maybe_unused]] Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { - return setPassword(user_name, "", session, mt, address); + return setPassword(user_name, "", session, address); } AuthenticationType getType() const override @@ -866,7 +855,7 @@ public: if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE) { std::unique_ptr password = mt.receive(); - return setPassword(user_name, password->password, session, mt, address); + return setPassword(user_name, password->password, session, address); } else throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, @@ -901,20 +890,30 @@ public: Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - const AuthenticationType user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); - if (type_to_method.find(user_auth_type) != type_to_method.end()) + AuthenticationType user_auth_type; + try { - type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); - mt.send(Messaging::AuthenticationOk(), true); - LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); - return; + user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); + if (type_to_method.find(user_auth_type) != type_to_method.end()) + { + type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); + mt.send(Messaging::AuthenticationOk(), true); + LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); + return; + } + } + catch (const Exception&) + { + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), + true); + + throw; } - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), - true); + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), + true); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication type {} is not supported.", user_auth_type); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication method is not supported: {}", user_auth_type); } }; } diff --git a/tests/config/users.d/session_log_test.xml b/tests/config/users.d/session_log_test.xml index daddaa6e4b9..cc2c2c5fcde 100644 --- a/tests/config/users.d/session_log_test.xml +++ b/tests/config/users.d/session_log_test.xml @@ -17,7 +17,7 @@ - + ::1 127.0.0.1 diff --git a/tests/queries/bugs/01747_system_session_log_long.reference b/tests/queries/0_stateless/01747_system_session_log_long.reference similarity index 73% rename from tests/queries/bugs/01747_system_session_log_long.reference rename to tests/queries/0_stateless/01747_system_session_log_long.reference index 9ecf7e05421..e4f0b6f6076 100644 --- a/tests/queries/bugs/01747_system_session_log_long.reference +++ b/tests/queries/0_stateless/01747_system_session_log_long.reference @@ -4,215 +4,291 @@ TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - No profiles no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles, no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles and two simple roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # plaintext_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # sha256_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # double_sha1_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_session_log_test_xml_user TCP LoginFailure 1 -invalid_session_log_test_xml_user HTTP LoginFailure 1 +invalid_session_log_test_xml_user HTTP LoginFailure many invalid_session_log_test_xml_user MySQL LoginFailure many +invalid_session_log_test_xml_user PostgreSQL LoginFailure many session_log_test_xml_user TCP LoginSuccess 1 session_log_test_xml_user TCP Logout 1 -session_log_test_xml_user HTTP LoginSuccess 1 -session_log_test_xml_user HTTP Logout 1 +session_log_test_xml_user HTTP LoginSuccess many +session_log_test_xml_user HTTP Logout many session_log_test_xml_user MySQL LoginSuccess 1 session_log_test_xml_user MySQL Logout 1 diff --git a/tests/queries/bugs/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh similarity index 78% rename from tests/queries/bugs/01747_system_session_log_long.sh rename to tests/queries/0_stateless/01747_system_session_log_long.sh index 9b127e0b48d..c6e93f4abd7 100755 --- a/tests/queries/bugs/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash # Tags: long, no-parallel, no-fasttest -# Tag no-fasttest: Accesses CH via mysql table function (which is unavailable) ################################################################################################## # Verify that login, logout, and login failure events are properly stored in system.session_log @@ -11,9 +10,8 @@ # Using multiple protocols # * native TCP protocol with CH client # * HTTP with CURL -# * MySQL - CH server accesses itself via mysql table function, query typically fails (unrelated) -# but auth should be performed properly. -# * PostgreSQL - CH server accesses itself via postgresql table function (currently out of order). +# * MySQL - CH server accesses itself via mysql table function. +# * PostgreSQL - CH server accesses itself via postgresql table function, but can't execute query (No LOGIN SUCCESS entry). # * gRPC - not done yet # # There is way to control how many time a query (e.g. via mysql table function) is retried @@ -53,7 +51,7 @@ function reportError() function executeQuery() { - ## Execute query (provided via heredoc or herestring) and print query in case of error. + # Execute query (provided via heredoc or herestring) and print query in case of error. trap 'rm -f ${TMP_QUERY_FILE}; trap - ERR RETURN' RETURN # Since we want to report with current values supplied to this function call # shellcheck disable=SC2064 @@ -82,7 +80,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} "${@}" --multiquery --queries-file "${TMP_QUERY_FILE}" 2>&1 | tee -a ${TMP_QUERY_FILE} + ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a ${TMP_QUERY_FILE} } function createUser() @@ -121,6 +119,8 @@ function createUser() executeQuery < Date: Fri, 28 Jul 2023 07:08:11 +0000 Subject: [PATCH 825/871] use same executor for GET_PART and ATTACH_PART --- 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 d62a1d960e6..2c2cea0af2b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3311,7 +3311,7 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne auto job_type = selected_entry->log_entry->type; /// Depending on entry type execute in fetches (small) pool or big merge_mutate pool - if (job_type == LogEntry::GET_PART) + if (job_type == LogEntry::GET_PART || job_type == LogEntry::ATTACH_PART) { assignee.scheduleFetchTask(std::make_shared( [this, selected_entry] () mutable From 63b05da1f2da6cee086d1154ddc670329aba667d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 07:23:34 +0000 Subject: [PATCH 826/871] System logs improvements --- .../settings.md | 128 ++++++++++- docs/en/operations/system-tables/index.md | 4 + .../settings.md | 212 +++++++++++++++--- docs/ru/operations/system-tables/index.md | 4 + programs/server/config.xml | 53 +++++ src/Common/SystemLogBase.cpp | 81 ++++--- src/Common/SystemLogBase.h | 43 +++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 4 + .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 3 + src/Interpreters/CrashLog.cpp | 5 +- src/Interpreters/CrashLog.h | 5 + src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/ProcessorsProfileLog.cpp | 7 - src/Interpreters/ProcessorsProfileLog.h | 7 +- src/Interpreters/Session.cpp | 2 +- src/Interpreters/SessionLog.cpp | 6 +- src/Interpreters/SystemLog.cpp | 92 +++++--- src/Interpreters/SystemLog.h | 17 +- src/Interpreters/TextLog.cpp | 11 +- src/Interpreters/TextLog.h | 13 +- src/Interpreters/ThreadStatusExt.cpp | 4 +- src/Interpreters/TraceCollector.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Interpreters/TransactionsInfoLog.cpp | 2 +- src/Loggers/Loggers.cpp | 44 +++- src/Loggers/OwnSplitChannel.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- tests/integration/parallel_skip.json | 9 +- .../test_crash_log/configs/crash_log.xml | 16 ++ tests/integration/test_crash_log/test.py | 19 +- .../test_system_flush_logs/test.py | 99 +++++++- .../test_system_logs/test_system_logs.py | 50 +++++ 40 files changed, 794 insertions(+), 178 deletions(-) create mode 100644 tests/integration/test_crash_log/configs/crash_log.xml diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a6ae517e401..e9f0f0dae00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -512,7 +512,7 @@ Both the cache for `local_disk`, and temporary data will be stored in `/tiny_loc cache local_disk /tiny_local_cache/ - 10M + 10M 1M 1 0 @@ -1592,6 +1592,10 @@ To manually turn on metrics history collection [`system.metric_log`](../../opera metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1695,6 +1699,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1706,6 +1718,10 @@ Use the following parameters to configure logging: part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1773,6 +1789,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1786,6 +1810,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1831,6 +1859,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size_rows, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1844,6 +1880,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1861,6 +1901,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1874,6 +1922,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1890,6 +1942,14 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1901,13 +1961,16 @@ Parameters: system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day ``` - ## trace_log {#server_configuration_parameters-trace_log} Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. @@ -1920,6 +1983,12 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1931,6 +2000,10 @@ The default server configuration file `config.xml` contains the following settin trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1945,9 +2018,18 @@ Parameters: - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) **Example** + ```xml @@ -1955,11 +2037,53 @@ Parameters: asynchronous_insert_log
7500 toYYYYMM(event_date) + 1048576 + 8192 + 524288 + false
``` +## crash_log {#server_configuration_parameters-crash_log} + +Settings for the [crash_log](../../operations/system-tables/crash-log.md) system table operation. + +Parameters: + +- `database` — Database for storing a table. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). + +The default server configuration file `config.xml` contains the following settings section: + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + false +
+``` + ## query_masking_rules {#query-masking-rules} Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs, diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 1b720098fc7..a46f306f677 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -47,6 +47,10 @@ An example: ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 421df3fe3eb..81a696bcfc1 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1058,6 +1058,10 @@ ClickHouse использует потоки из глобального пул metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1155,12 +1159,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. - +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ``` xml @@ -1169,6 +1180,10 @@ ClickHouse использует потоки из глобального пул part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1218,11 +1233,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1234,6 +1257,10 @@ ClickHouse использует потоки из глобального пул query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1245,11 +1272,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1261,6 +1296,10 @@ ClickHouse использует потоки из глобального пул query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1272,11 +1311,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` – имя базы данных. -- `table` – имя системной таблицы, где будут логироваться запросы. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`. -- `engine` — устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать, если задан параметр `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1288,6 +1335,10 @@ ClickHouse использует потоки из глобального пул query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1297,12 +1348,20 @@ ClickHouse использует потоки из глобального пул Параметры: -- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. -- `database` — имя базы данных для хранения таблицы. -- `table` — имя таблицы, куда будут записываться текстовые сообщения. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ```xml @@ -1312,6 +1371,10 @@ ClickHouse использует потоки из глобального пул system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day @@ -1323,13 +1386,21 @@ ClickHouse использует потоки из глобального пул Настройки для [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. -Parameters: +Параметры: -- `database` — Database for storing a table. -- `table` — Table name. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. По умолчанию файл настроек сервера `config.xml` содержит следующие настройки: @@ -1339,9 +1410,84 @@ Parameters: trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 ``` +## asynchronous_insert_log {#server_configuration_parameters-asynchronous_insert_log} + +Настройки для asynchronous_insert_log Система для логирования ассинхронных вставок. + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. + +**Пример** + +```xml + + + system + asynchronous_insert_log
+ 7500 + toYYYYMM(event_date) + 1048576 + 8192 + 524288 + +
+
+``` + +## crash_log {#server_configuration_parameters-crash_log} + +Настройки для таблицы [crash_log](../../operations/system-tables/crash-log.md). + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1024. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 1024. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: true. + +**Пример** + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + true +
+``` + ## query_masking_rules {#query-masking-rules} Правила, основанные на регулярных выражениях, которые будут применены для всех запросов, а также для всех сообщений перед сохранением их в лог на сервере, diff --git a/docs/ru/operations/system-tables/index.md b/docs/ru/operations/system-tables/index.md index 7ff368b1910..24f79cae212 100644 --- a/docs/ru/operations/system-tables/index.md +++ b/docs/ru/operations/system-tables/index.md @@ -45,6 +45,10 @@ sidebar_label: "Системные таблицы" ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/programs/server/config.xml b/programs/server/config.xml index 2a7dc1e576a..153cb728bb4 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1026,6 +1026,14 @@ 7500 + + 1048576 + + 8192 + + 524288 + + false @@ -1039,6 +1047,11 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + + false @@ -1084,7 +1109,11 @@ system metric_log
7500 + 1048576 + 8192 + 524288 1000 + false @@ -1151,6 +1196,10 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false + in_order + + 2 + 4 + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 500 + + diff --git a/tests/integration/test_zookeeper_fallback_session/test.py b/tests/integration/test_zookeeper_fallback_session/test.py new file mode 100644 index 00000000000..570eca4f0a6 --- /dev/null +++ b/tests/integration/test_zookeeper_fallback_session/test.py @@ -0,0 +1,101 @@ +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager + + +cluster = ClickHouseCluster( + __file__, zookeeper_config_path="configs/zookeeper_load_balancing.xml" +) + +node1 = cluster.add_instance( + "node1", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) +node2 = cluster.add_instance( + "node2", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) +node3 = cluster.add_instance( + "node3", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + for node in [node1, node2, node3]: + node.query("DROP TABLE IF EXISTS simple SYNC") + node.query( + """ + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; + """.format( + replica=node.name + ) + ) + yield cluster + finally: + cluster.shutdown() + + +def assert_uses_zk_node(node: ClickHouseInstance, zk_node): + def check_callback(host): + return host.strip() == expected_zk_ip_addr + + expected_zk_ip_addr = node.cluster.get_instance_ip(zk_node) + + host = node.query_with_retry( + "select host from system.zookeeper_connection", check_callback=check_callback + ) + assert host.strip() == expected_zk_ip_addr + + +def test_fallback_session(started_cluster: ClickHouseCluster): + # only leave connecting to zoo3 possible + with PartitionManager() as pm: + for node in started_cluster.instances.values(): + for zk in ["zoo1", "zoo2"]: + pm._add_rule( + { + "source": node.ip_address, + "destination": cluster.get_instance_ip(zk), + "action": "REJECT --reject-with tcp-reset", + } + ) + + for node in [node1, node2, node3]: + # all nodes will have to switch to zoo3 + assert_uses_zk_node(node, "zoo3") + + node1.query_with_retry("INSERT INTO simple VALUES ({0}, {0})".format(1)) + + # and replication still works + for node in [node2, node3]: + assert ( + node.query_with_retry( + "SELECT count() from simple", + check_callback=lambda count: count.strip() == "1", + ) + == "1\n" + ) + + # at this point network partitioning has been reverted. + # the nodes should switch to zoo1 automatically because of `in_order` load-balancing. + # otherwise they would connect to a random replica + for node in [node1, node2, node3]: + assert_uses_zk_node(node, "zoo1") + + node1.query_with_retry("INSERT INTO simple VALUES ({0}, {0})".format(2)) + for node in [node2, node3]: + assert ( + node.query_with_retry( + "SELECT count() from simple", + check_callback=lambda count: count.strip() == "2", + ) + == "2\n" + ) From caef499541d2ecdc69354b7444291aa5f44f3482 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Jul 2023 12:37:51 +0000 Subject: [PATCH 829/871] fix test --- tests/queries/0_stateless/02833_multiprewhere_extra_column.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql index a786de454ed..3a751294cba 100644 --- a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings +-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings, no-s3-storage drop table if exists t_multi_prewhere; drop row policy if exists policy_02834 on t_multi_prewhere; From ff9e85529ad638f5717670294cfc00eb6aa19140 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 28 Jul 2023 12:20:48 +0000 Subject: [PATCH 830/871] Allow reading zero objects in CachedObjectStorage::readObjects() --- .../Cached/CachedObjectStorage.cpp | 2 - .../02731_zero_objects_in_metadata.reference | 3 ++ .../02731_zero_objects_in_metadata.sh | 37 ++++++++++--------- 3 files changed, 23 insertions(+), 19 deletions(-) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 3e7c4d12c42..717acb08f62 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -79,8 +79,6 @@ std::unique_ptr CachedObjectStorage::readObjects( /// NO std::optional read_hint, std::optional file_size) const { - if (objects.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Received empty list of objects to read"); return object_storage->readObjects(objects, patchSettings(read_settings), read_hint, file_size); } diff --git a/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference b/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference index 125915f4f65..1898b2c543b 100644 --- a/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference +++ b/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference @@ -1,3 +1,6 @@ 1 [] 1 [] [] +1 [] +1 [] +[] diff --git a/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh b/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh index 690cf977d08..eef52002e36 100755 --- a/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh +++ b/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh @@ -5,23 +5,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " -DROP TABLE IF EXISTS test; -CREATE TABLE test (id Int32, empty Array(Int32)) - ENGINE=MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='s3_disk'; +for DISK in s3_disk s3_cache +do + ${CLICKHOUSE_CLIENT} -n --query " + DROP TABLE IF EXISTS test; + CREATE TABLE test (id Int32, empty Array(Int32)) + ENGINE=MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='$DISK'; -INSERT INTO test (id) VALUES (1); -SELECT * FROM test; -" + INSERT INTO test (id) VALUES (1); + SELECT * FROM test; + " -${CLICKHOUSE_CLIENT} -n --query " -BACKUP TABLE test TO Disk('backups', 'test_s3_backup'); -DROP TABLE test; -RESTORE TABLE test FROM Disk('backups', 'test_s3_backup'); -" &>/dev/null + ${CLICKHOUSE_CLIENT} -n --query " + BACKUP TABLE test TO Disk('backups', 'test_s3_backup'); + DROP TABLE test; + RESTORE TABLE test FROM Disk('backups', 'test_s3_backup'); + " &>/dev/null -${CLICKHOUSE_CLIENT} -n --query " -SELECT * FROM test; -SELECT empty FROM test; -" + ${CLICKHOUSE_CLIENT} -n --query " + SELECT * FROM test; + SELECT empty FROM test; + " +done From 139033247cad342dbdf1f58428921328d5c191aa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 28 Jul 2023 15:46:45 +0200 Subject: [PATCH 831/871] bypass a limitation of github From bf9504d95d65f23552ac5708c2d0365125269085 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 28 Jul 2023 15:49:55 +0200 Subject: [PATCH 832/871] Move UnlinkMetadataFileOperationOutcome to common header (#52710) --- src/Disks/ObjectStorages/IMetadataStorage.h | 9 ++++++++- .../MetadataStorageFromDiskTransactionOperations.h | 9 --------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 264c481ee08..6b75e157dee 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -22,7 +22,14 @@ namespace ErrorCodes } class IMetadataStorage; -struct UnlinkMetadataFileOperationOutcome; + +/// Return the result of operation to the caller. +/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal. +struct UnlinkMetadataFileOperationOutcome +{ + UInt32 num_hardlinks = std::numeric_limits::max(); +}; + using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; /// Tries to provide some "transactions" interface, which allow diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index 4662ebc3423..ccb77f6ae7b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -244,15 +244,6 @@ private: std::unique_ptr write_operation; }; -/// Return the result of operation to the caller. -/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal. -struct UnlinkMetadataFileOperationOutcome -{ - UInt32 num_hardlinks = std::numeric_limits::max(); -}; - -using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; - struct UnlinkMetadataFileOperation final : public IMetadataOperation { const UnlinkMetadataFileOperationOutcomePtr outcome = std::make_shared(); From 7cb853bcd89746c454ac46095bd4e75b155ce1e1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 28 Jul 2023 14:22:12 +0000 Subject: [PATCH 833/871] Fix TableFunctionNode::cloneImpl --- src/Analyzer/TableFunctionNode.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 30644ad4ec4..e5158a06373 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -133,6 +133,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const result->storage_snapshot = storage_snapshot; result->table_expression_modifiers = table_expression_modifiers; result->settings_changes = settings_changes; + result->unresolved_arguments_indexes = unresolved_arguments_indexes; return result; } From cecc6364496333d52a3d6a7aa6bd1ba55678c2f4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 Jul 2023 19:35:22 +0200 Subject: [PATCH 834/871] Update CachedObjectStorage.cpp --- src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 717acb08f62..298cffaf9b0 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -16,11 +16,6 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - CachedObjectStorage::CachedObjectStorage( ObjectStoragePtr object_storage_, FileCachePtr cache_, From 2a40bc9055d446d0d9f04d25b47c96d19f8d37fe Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:29:59 +0000 Subject: [PATCH 835/871] Fix: __bitSwapLastTwo was applied to non BoolMask - ActionsDAG was missing convertions to BoolMask - __bitWrapperFunc --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 9c34a149128..56550ab7f3a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -276,7 +277,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Filter actions DAG:\n{}", filter_actions_dag->dumpDAG()); actions = std::make_shared(filter_actions_dag); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } else { @@ -290,7 +293,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( traverseAST(expression_ast); auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList()); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "expression AST:\n{}", expression_ast->dumpTree()); actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } } @@ -351,6 +356,8 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA const ContextPtr & context, std::unordered_map & node_to_result_node) const { + // LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "Traversing node:\n{}", node.dumpDAG()); + auto result_node_it = node_to_result_node.find(&node); if (result_node_it != node_to_result_node.end()) return *result_node_it->second; @@ -457,8 +464,11 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio if (arguments_size != 1) return nullptr; + auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); + const auto & bit_wrapper_func_node = result_dag->addFunction(bit_wrapper_function, {arguments[0]}, {}); + auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); - return &result_dag->addFunction(bit_swap_last_two_function, {arguments[0]}, {}); + return &result_dag->addFunction(bit_swap_last_two_function, {&bit_wrapper_func_node}, {}); } else if (function_name == "and" || function_name == "indexHint" || function_name == "or") { From 45b96142249c02f71b140b14784969b9a228760c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:41:18 +0000 Subject: [PATCH 836/871] Remove debug logging --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 56550ab7f3a..05ea7d15603 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -277,9 +276,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Filter actions DAG:\n{}", filter_actions_dag->dumpDAG()); actions = std::make_shared(filter_actions_dag); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } else { @@ -293,9 +290,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( traverseAST(expression_ast); auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList()); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "expression AST:\n{}", expression_ast->dumpTree()); actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } } @@ -356,8 +351,6 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA const ContextPtr & context, std::unordered_map & node_to_result_node) const { - // LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "Traversing node:\n{}", node.dumpDAG()); - auto result_node_it = node_to_result_node.find(&node); if (result_node_it != node_to_result_node.end()) return *result_node_it->second; From a6fc632aa18f45ad141d3f524bf1492d0450060a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:57:56 +0000 Subject: [PATCH 837/871] More test queries + update file with broken tests --- tests/analyzer_tech_debt.txt | 1 - tests/queries/0_stateless/00979_set_index_not.reference | 2 ++ tests/queries/0_stateless/00979_set_index_not.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 737e0e0a5e4..c8f2bb9f43d 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -11,7 +11,6 @@ 00927_asof_joins 00940_order_by_read_in_order_query_plan 00945_bloom_filter_index -00979_set_index_not 00981_in_subquery_with_tuple 01049_join_low_card_bug_long 01062_pm_all_join_with_block_continuation diff --git a/tests/queries/0_stateless/00979_set_index_not.reference b/tests/queries/0_stateless/00979_set_index_not.reference index 455708dfe99..033fcb0467b 100644 --- a/tests/queries/0_stateless/00979_set_index_not.reference +++ b/tests/queries/0_stateless/00979_set_index_not.reference @@ -1,2 +1,4 @@ Jon alive Jon alive +Ramsey rip +Ramsey rip diff --git a/tests/queries/0_stateless/00979_set_index_not.sql b/tests/queries/0_stateless/00979_set_index_not.sql index 2ad27e337f9..13a0b4cbb09 100644 --- a/tests/queries/0_stateless/00979_set_index_not.sql +++ b/tests/queries/0_stateless/00979_set_index_not.sql @@ -11,5 +11,7 @@ insert into set_index_not values ('Jon','alive'),('Ramsey','rip'); select * from set_index_not where status!='rip'; select * from set_index_not where NOT (status ='rip'); +select * from set_index_not where NOT (status!='rip'); +select * from set_index_not where NOT (NOT (status ='rip')); DROP TABLE set_index_not; From 553304a81ac0063f4d60d178ef9b1763fd2b083c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 29 Jul 2023 11:15:56 +0200 Subject: [PATCH 838/871] Fixed clang tidy build by removing unued variable --- src/Storages/StorageSnapshot.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 6abca59268f..c0e85900794 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -120,9 +120,8 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons const auto & columns = getMetadataForQuery()->getColumns(); for (const auto & column_name : column_names) { - std::string substituted_column_name = column_name; - auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); - auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); + auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); + auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); if (column && !object_column) { res.insert({column->type->createColumn(), column->type, column_name}); @@ -141,7 +140,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons else { throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, - "Column {} not found in table {}", backQuote(substituted_column_name), storage.getStorageID().getNameForLogs()); + "Column {} not found in table {}", backQuote(column_name), storage.getStorageID().getNameForLogs()); } } return res; From 84ab4238426b96fb126be8c59055657ed104c1c2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 28 Jul 2023 18:18:38 +0200 Subject: [PATCH 839/871] Allow reading empty file with no blobs --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 762151b3808..6bb6982065a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -485,8 +486,15 @@ std::unique_ptr DiskObjectStorage::readFile( std::optional read_hint, std::optional file_size) const { + auto storage_objects = metadata_storage->getStorageObjects(path); + + const bool file_can_be_empty = !file_size.has_value() || *file_size == 0; + + if (storage_objects.empty() && file_can_be_empty) + return std::make_unique(); + return object_storage->readObjects( - metadata_storage->getStorageObjects(path), + storage_objects, object_storage->getAdjustedSettingsFromMetadataFile(settings, path), read_hint, file_size); From cbb546a1a97b4d9819efa82547b48103a40d575d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 00:18:08 +0300 Subject: [PATCH 840/871] Update easy_tasks_sorted_ru.md --- tests/instructions/easy_tasks_sorted_ru.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/instructions/easy_tasks_sorted_ru.md b/tests/instructions/easy_tasks_sorted_ru.md index 09ea48d0bd9..17e9708eef5 100644 --- a/tests/instructions/easy_tasks_sorted_ru.md +++ b/tests/instructions/easy_tasks_sorted_ru.md @@ -129,7 +129,7 @@ position с конца строки. Атомарно удаляет таблицу перед созданием новой, если такая была. -## * Приведение типов для IN (subquery). +## + Приведение типов для IN (subquery). `SELECT 1 IN (SELECT -1 UNION ALL SELECT 1)` @@ -205,12 +205,12 @@ https://clickhouse.com/docs/en/operations/table_engines/external_data/ ## Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации. -## Настройка rollup_use_nulls. +## + Настройка rollup_use_nulls. + +Upd: it is named "group_by_use_nulls". ## + Настройка cast_keep_nullable. -## Функция bitEquals для сравнения произвольных типов данных побитово. +## Функция bitEquals для сравнения произвольных типов данных побитово ## Функция serialize для implementation specific non portable non backwards compatible сериализации любого типа данных в набор байт. - -## Функция bitEquals и оператор <=>. From 63b155a74111e7f2b1b94808e8f9aa47374bc71f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 23:12:54 +0000 Subject: [PATCH 841/871] bypass merging attempt for non 'clickhouse' configurations with different root node names --- src/Common/Config/ConfigProcessor.cpp | 13 +++++++++++-- src/Common/Config/ConfigProcessor.h | 4 +++- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index a55183782d8..04f55600b40 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -328,7 +328,7 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, } } -void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) +bool ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) { Node * config_root = getRootNode(config.get()); Node * with_root = getRootNode(with.get()); @@ -343,11 +343,15 @@ void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) && !((config_root_node_name == "yandex" || config_root_node_name == "clickhouse") && (merged_root_node_name == "yandex" || merged_root_node_name == "clickhouse"))) { + if (config_root_node_name != "clickhouse" && config_root_node_name != "yandex") + return false; + throw Poco::Exception("Root element doesn't have the corresponding root element as the config file." " It must be <" + config_root->nodeName() + ">"); } mergeRecursive(config, config_root, with_root); + return true; } void ConfigProcessor::doIncludesRecursive( @@ -645,7 +649,12 @@ XMLDocumentPtr ConfigProcessor::processConfig( with = dom_parser.parse(merge_file); } - merge(config, with); + if (!merge(config, with)) + { + LOG_DEBUG(log, "Merging bypassed - configuration file '{}' doesn't belong to configuration '{}' - merging root node name '{}' doesn't match '{}'", + merge_file, path, getRootNode(with.get())->nodeName(), getRootNode(config.get())->nodeName()); + continue; + } contributing_files.push_back(merge_file); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 060ef49d36a..b4f85b10526 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -144,7 +144,9 @@ private: void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); - void merge(XMLDocumentPtr config, XMLDocumentPtr with); + /// If config root node name is not 'clickhouse' and merging config's root node names doesn't match, bypasses merging and returns false. + /// For compatibility root node 'yandex' considered equal to 'clickhouse'. + bool merge(XMLDocumentPtr config, XMLDocumentPtr with); void doIncludesRecursive( XMLDocumentPtr config, From 4d27cf4386e1b3b3d0b1516723fbef0f9143ebcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 02:11:32 +0200 Subject: [PATCH 842/871] Proper stripping --- cmake/split_debug_symbols.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index 8ba7669d9e2..d6821eb6c48 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -24,7 +24,7 @@ macro(clickhouse_split_debug_symbols) COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check. # Also, after we disabled the export of symbols for dynamic linking, we still to keep a static symbol table for good stack traces. - COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash --keep-section=.symtab --keep-section=.strtab --keep-section=.shstrtab "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" + COMMAND "${STRIP_PATH}" --strip-debug --remove-section=.comment --remove-section=.note "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" # Associate stripped binary with debug symbols: COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" COMMENT "Stripping clickhouse binary" VERBATIM From a4f2eea51603bb98340e8c6add215ca8b7fc676d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 02:19:39 +0200 Subject: [PATCH 843/871] Maybe better --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 67c185a0b54..b728602ef40 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -12,6 +12,7 @@ # (simple regexps) to check if the code is likely to have basic style violations. # and then to run formatter only for the specified files. +LC_ALL="en_US.UTF-8" ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml' From 1deebd3ae624232e2b9ae05e7b1350816ef490ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 04:22:36 +0200 Subject: [PATCH 844/871] Fix Docker --- docker/test/base/Dockerfile | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index f6836804454..bfbe9461c1e 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -11,6 +11,7 @@ RUN apt-get update \ pv \ ripgrep \ zstd \ + locales \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) @@ -28,7 +29,13 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_de ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' +RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 +ENV LC_ALL en_US.UTF-8 + ENV TZ=Europe/Moscow RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 + + +RUN apt-get update && apt-get install -y locales From a336f06f57542015ba63043a4bdf7261ee1f0ac3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:46:05 +0300 Subject: [PATCH 845/871] Update Context.cpp --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 123c2ab8f85..8b8333524b8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1581,7 +1581,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); - StorageView::replaceQueryParametersIfParametrizedView(query,parameterized_view_values); + StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; create.select = query->as(); From 972653cde3045d309c2bff180630cedefa9b10b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:18:52 +0200 Subject: [PATCH 846/871] Change the default timezones in Docker test images --- docker/test/base/Dockerfile | 2 +- docker/test/fasttest/Dockerfile | 2 +- docker/test/fuzzer/Dockerfile | 2 +- docker/test/performance-comparison/Dockerfile | 2 +- docker/test/stateless/Dockerfile | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index f6836804454..17aed0d4402 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -28,7 +28,7 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_de ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index da4baa8c687..ad24e662a6c 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -32,7 +32,7 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV COMMIT_SHA='' diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index aa71074c02a..0bc0fb06633 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -8,7 +8,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ENV LANG=C.UTF-8 -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN apt-get update \ diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index ab9f1f8a2e3..cfd7c613868 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -11,7 +11,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ENV LANG=C.UTF-8 -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN apt-get update \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index e1e84c427ba..6f44c7cc88b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -52,7 +52,7 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV NUM_TRIES=1 From ecfde7edc6ad3292739ada60ec36c8f9413f9ffe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:20:51 +0200 Subject: [PATCH 847/871] Fix Docker --- docker/test/style/Dockerfile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 2aa0b1a62d6..bd1c26855b7 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,9 +18,13 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ + locales \ && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ - && rm -rf /root/.cache/pip + && rm -rf /root/.cache/pip + +RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 +ENV LC_ALL en_US.UTF-8 # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH From 424fe43090fce65d564997452a85e5d617271b52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 06:57:11 +0300 Subject: [PATCH 848/871] Update Dockerfile --- docker/test/base/Dockerfile | 3 --- 1 file changed, 3 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index bfbe9461c1e..6ba44662af9 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -36,6 +36,3 @@ ENV TZ=Europe/Moscow RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 - - -RUN apt-get update && apt-get install -y locales From 85387912ee3c1f5345b8c5b95181e1325124eb6d Mon Sep 17 00:00:00 2001 From: xiebin Date: Sun, 30 Jul 2023 12:06:30 +0800 Subject: [PATCH 849/871] just remove the first bad query, because automatically converting the layout simple to complex with two keys now. --- .../01018_ddl_dictionaries_bad_queries.reference | 1 - .../01018_ddl_dictionaries_bad_queries.sh | 15 --------------- 2 files changed, 16 deletions(-) diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference index abc3218ce6c..5ee4e7592f6 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference @@ -4,4 +4,3 @@ 2 2 2 -2 diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh index 6826ed677db..8a40129ad4f 100755 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh @@ -9,21 +9,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1" -# Simple layout, but with two keys -$CLICKHOUSE_CLIENT -q " - CREATE DICTIONARY dict1 - ( - key1 UInt64, - key2 UInt64, - value String - ) - PRIMARY KEY key1, key2 - LAYOUT(HASHED()) - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) - LIFETIME(MIN 1 MAX 10) -" 2>&1 | grep -c 'Primary key for simple dictionary must contain exactly one element' - - # Simple layout, but with non existing key $CLICKHOUSE_CLIENT -q " CREATE DICTIONARY dict1 From 41ffc097711b9ce3c3a26ba0fdc38315e3a39363 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 09:18:06 +0200 Subject: [PATCH 850/871] Upload build statistics to the CI database --- tests/ci/build_check.py | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 35b98a7c3bb..9e44af5296c 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -35,6 +35,11 @@ from version_helper import ( get_version_from_repo, update_version_local, ) +from clickhouse_helper import ( + ClickHouseHelper, + prepare_tests_results_for_clickhouse, +) +from stopwatch import Stopwatch IMAGE_NAME = "clickhouse/binary-builder" BUILD_LOG_NAME = "build_log.log" @@ -268,6 +273,7 @@ def mark_failed_reports_pending(build_name: str, pr_info: PRInfo) -> None: def main(): logging.basicConfig(level=logging.INFO) + stopwatch = Stopwatch() build_name = sys.argv[1] build_config = CI_CONFIG["build_config"][build_name] @@ -394,7 +400,20 @@ def main(): ) upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) - # Fail build job if not successeded + + ch_helper = ClickHouseHelper() + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [], + "success" if success else "failure", + stopwatch.duration_seconds, + stopwatch.start_time_str, + log_url, + f"Build ({build_name})", + ) + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + # Fail the build job if it didn't succeed if not success: sys.exit(1) From 448756d14872cc89cc08aedf80e214941d1fdd4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 09:31:21 +0200 Subject: [PATCH 851/871] Add machine_type information to the CI database --- tests/ci/clickhouse_helper.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 9410b37d69f..eca5f6f399d 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -132,6 +132,21 @@ class ClickHouseHelper: return result +# Obtain the machine type from IMDS: +def get_instance_type(): + url = "http://169.254.169.254/latest/meta-data/instance-type" + for i in range(5): + try: + response = requests.get(url, timeout=1) + if response.status_code == 200: + return response.text + except Exception as e: + error = f"Received exception while sending data to {url} on {i} attempt: {e}" + logging.warning(error) + continue + return '' + + def prepare_tests_results_for_clickhouse( pr_info: PRInfo, test_results: TestResults, @@ -168,6 +183,7 @@ def prepare_tests_results_for_clickhouse( head_ref=head_ref, head_repo=head_repo, task_url=pr_info.task_url, + instance_type=get_instance_type(), ) # Always publish a total record for all checks. For checks with individual From 358773c620ad091458c63de1e37307a053f044c9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 30 Jul 2023 07:45:27 +0000 Subject: [PATCH 852/871] Automatic style fix --- tests/ci/clickhouse_helper.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index eca5f6f399d..7f98f6e52d2 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,10 +141,12 @@ def get_instance_type(): if response.status_code == 200: return response.text except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" + error = ( + f"Received exception while sending data to {url} on {i} attempt: {e}" + ) logging.warning(error) continue - return '' + return "" def prepare_tests_results_for_clickhouse( From bc1f80dd82d2992aa108f323af4349bd2f4c9968 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 10:25:44 +0200 Subject: [PATCH 853/871] Remove Coverity (part 2) --- docker/packager/binary/build.sh | 17 ----------------- docker/packager/packager | 7 +------ tests/ci/ci_config.py | 11 ----------- 3 files changed, 1 insertion(+), 34 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index c0803c74147..ce3f729fe33 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -80,16 +80,6 @@ else cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. fi -if [ "coverity" == "$COMBINED_OUTPUT" ] -then - mkdir -p /workdir/cov-analysis - - wget --post-data "token=$COVERITY_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /workdir/cov-analysis --strip-components 1 - export PATH=$PATH:/workdir/cov-analysis/bin - cov-configure --config ./coverity.config --template --comptype clangcc --compiler "$CC" - SCAN_WRAPPER="cov-build --config ./coverity.config --dir cov-int" -fi - # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. $SCAN_WRAPPER ninja $NINJA_FLAGS $BUILD_TARGET @@ -175,13 +165,6 @@ then mv "$COMBINED_OUTPUT.tar.zst" /output fi -if [ "coverity" == "$COMBINED_OUTPUT" ] -then - # Coverity does not understand ZSTD. - tar -cvz -f "coverity-scan.tar.gz" cov-int - mv "coverity-scan.tar.gz" /output -fi - ccache_status ccache --evict-older-than 1d diff --git a/docker/packager/packager b/docker/packager/packager index e12bd55dde3..399e12e6680 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -253,11 +253,6 @@ def parse_env_variables( cmake_flags.append(f"-DCMAKE_C_COMPILER={cc}") cmake_flags.append(f"-DCMAKE_CXX_COMPILER={cxx}") - # Create combined output archive for performance tests. - if package_type == "coverity": - result.append("COMBINED_OUTPUT=coverity") - result.append('COVERITY_TOKEN="$COVERITY_TOKEN"') - if sanitizer: result.append(f"SANITIZER={sanitizer}") if build_type: @@ -356,7 +351,7 @@ def parse_args() -> argparse.Namespace: ) parser.add_argument( "--package-type", - choices=["deb", "binary", "coverity"], + choices=["deb", "binary"], required=True, ) parser.add_argument( diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index db9a7f926be..bea654ca76f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -19,16 +19,6 @@ CI_CONFIG = { "with_coverage": False, "comment": "", }, - "coverity": { - "compiler": "clang-16", - "build_type": "", - "sanitizer": "", - "package_type": "coverity", - "tidy": "disable", - "with_coverage": False, - "official": False, - "comment": "A special build for coverity", - }, "package_aarch64": { "compiler": "clang-16-aarch64", "build_type": "", @@ -187,7 +177,6 @@ CI_CONFIG = { "builds_report_config": { "ClickHouse build check": [ "package_release", - "coverity", "package_aarch64", "package_asan", "package_ubsan", From 6bff19bd824bb38d026b12b000a76e7b47c098f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 10:27:33 +0200 Subject: [PATCH 854/871] Remove Coverity (part 2) --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ce3f729fe33..640654e5445 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -82,7 +82,7 @@ fi # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. -$SCAN_WRAPPER ninja $NINJA_FLAGS $BUILD_TARGET +ninja $NINJA_FLAGS $BUILD_TARGET ls -la ./programs From 08ef4d015c6b42061f6de467279f0f1207913435 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 30 Jul 2023 13:36:52 +0200 Subject: [PATCH 855/871] Minor change --- docs/en/operations/system-tables/events.md | 2 +- docs/en/operations/system-tables/metrics.md | 2 +- src/Storages/System/StorageSystemEvents.h | 1 + src/Storages/System/StorageSystemMetrics.h | 1 + 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index b53628a8384..26f5cb2031b 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -10,7 +10,7 @@ Columns: - `event` ([String](../../sql-reference/data-types/string.md)) — Event name. - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. -- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for event. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `event`. **Example** diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index ea36411cf3d..6771e8d7aad 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -10,7 +10,7 @@ Columns: - `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. -- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for metric. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `metric`. The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index ed80a441529..b2e4bac072a 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -18,6 +18,7 @@ public: static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index bffc7e6f1fc..e3e2c07014f 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -19,6 +19,7 @@ public: static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; From aba291ca688fa58d82efc329e7be6d1c2e638f4a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 30 Jul 2023 14:01:34 +0200 Subject: [PATCH 856/871] Revert "Revert "Add an ability to specify allocations size for sampling memory profiler"" --- programs/server/Server.cpp | 21 ++++++++--- src/Common/MemoryTracker.cpp | 10 ++++- src/Common/MemoryTracker.h | 18 +++++++++ src/Core/ServerSettings.h | 8 +++- src/Core/Settings.h | 4 +- src/Interpreters/ProcessList.cpp | 3 ++ src/Interpreters/ThreadStatusExt.cpp | 2 + .../__init__.py | 1 + .../configs/max_untracked_memory.xml | 7 ++++ .../configs/memory_profiler.xml | 5 +++ .../test.py | 37 +++++++++++++++++++ ...r_sample_min_max_allocation_size.reference | 1 + ...profiler_sample_min_max_allocation_size.sh | 18 +++++++++ 13 files changed, 124 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py create mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference create mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 405ebf7fb2f..85ee48ea075 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1691,17 +1691,26 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); - if (total_memory_profiler_step) + if (server_settings.total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(total_memory_profiler_step); + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); } - double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); - if (total_memory_tracker_sample_probability > 0.0) + if (server_settings.total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 81cac2617c5..52cae0768dc 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,6 +534,12 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } +bool MemoryTracker::isSizeOkForSampling(UInt64 size) const +{ + /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation + return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); +} + bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 3ea1ea8702c..6f576e53e7a 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,6 +67,12 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; + /// Randomly sample allocations only larger or equal to this size + UInt64 min_allocation_size_bytes = 0; + + /// Randomly sample allocations only smaller or equal to this size + UInt64 max_allocation_size_bytes = 0; + /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -88,6 +94,8 @@ private: void setOrRaiseProfilerLimit(Int64 value); + bool isSizeOkForSampling(UInt64 size) const; + /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -166,6 +174,16 @@ public: sample_probability = value; } + void setSampleMinAllocationSize(UInt64 value) + { + min_allocation_size_bytes = value; + } + + void setSampleMaxAllocationSize(UInt64 value) + { + max_allocation_size_bytes = value; + } + void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index ca27cbdbf19..7678e8c3f24 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -83,8 +83,12 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) - + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ + \ + M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ + M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..26e0808aaa3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -427,7 +427,9 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1503e396298..c299572ef41 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); + thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7a6bc45c118..1b39bb6c0dc 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml new file mode 100644 index 00000000000..56fc5ed34ca --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml new file mode 100644 index 00000000000..5b3e17d145f --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml @@ -0,0 +1,5 @@ + + 1 + 4096 + 8192 + diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py new file mode 100644 index 00000000000..6ab971fa9c4 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -0,0 +1,37 @@ +from helpers.cluster import ClickHouseCluster +import pytest + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/memory_profiler.xml"], + user_configs=["configs/max_untracked_memory.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_trace_boundaries_work(started_cluster): + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") + node.query("SYSTEM FLUSH LOGS") + + assert ( + node.query( + "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" + ) + == "0\n" + ) diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh new file mode 100755 index 00000000000..b1fbea26da7 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings +# requires TraceCollector, does not available under sanitizers and aarch64 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" +${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + +# at least something allocated +${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" + +# show wrong allocations +${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From 3e5767853d824034221b981ef7c3ad29f0e0ce77 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 30 Jul 2023 14:15:17 +0200 Subject: [PATCH 857/871] Disable test for sanitizers --- tests/integration/test_memory_profiler_min_max_borders/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py index 6ab971fa9c4..df9439bc2bb 100644 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -20,6 +20,9 @@ def started_cluster(): def test_trace_boundaries_work(started_cluster): + if node.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") node.query("SYSTEM FLUSH LOGS") From e58b3cfd3246681092908dc6e38d0b9a130b5df7 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 30 Jul 2023 22:09:03 +0000 Subject: [PATCH 858/871] Implemented max sessions for user --- .../operations/settings/query-complexity.md | 36 +++ .../operations/settings/settings-profiles.md | 4 +- .../operations/settings/query-complexity.md | 37 +++ .../operations/settings/settings-profiles.md | 3 +- src/Access/ContextAccess.cpp | 32 ++- src/Access/SettingsConstraints.cpp | 81 +++++-- src/Access/SettingsConstraints.h | 23 +- src/Client/Suggest.cpp | 3 + src/Common/ErrorCodes.cpp | 1 + src/Common/SettingSource.h | 43 ++++ src/Core/Settings.h | 2 + .../Access/InterpreterCreateRoleQuery.cpp | 2 +- .../InterpreterCreateSettingsProfileQuery.cpp | 2 +- .../Access/InterpreterCreateUserQuery.cpp | 2 +- src/Interpreters/Context.cpp | 37 ++- src/Interpreters/Context.h | 17 +- src/Interpreters/InterpreterSetQuery.cpp | 4 +- src/Interpreters/Session.cpp | 39 ++- src/Interpreters/Session.h | 3 + src/Interpreters/SessionTracker.cpp | 62 +++++ src/Interpreters/SessionTracker.h | 60 +++++ src/Server/GRPCServer.cpp | 4 +- src/Server/HTTPHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 21 +- tests/integration/parallel_skip.json | 12 +- .../__init__.py | 0 .../configs/dhparam.pem | 8 + .../configs/log.xml | 9 + .../configs/ports.xml | 9 + .../configs/server.crt | 18 ++ .../configs/server.key | 28 +++ .../configs/ssl_conf.xml | 17 ++ .../configs/users.xml | 16 ++ .../protos/clickhouse_grpc.proto | 1 + .../test.py | 222 ++++++++++++++++++ ...2832_alter_max_sessions_for_user.reference | 12 + .../02832_alter_max_sessions_for_user.sh | 64 +++++ 37 files changed, 851 insertions(+), 85 deletions(-) create mode 100644 src/Common/SettingSource.h create mode 100644 src/Interpreters/SessionTracker.cpp create mode 100644 src/Interpreters/SessionTracker.h create mode 100755 tests/integration/test_profile_max_sessions_for_user/__init__.py create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/log.xml create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/ports.xml create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/server.crt create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/server.key create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/users.xml create mode 120000 tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto create mode 100755 tests/integration/test_profile_max_sessions_for_user/test.py create mode 100644 tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference create mode 100755 tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 163ed5d5826..3af368b87a7 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -327,3 +327,39 @@ The maximum amount of data consumed by temporary files on disk in bytes for all Zero means unlimited. Default value: 0. + +## max_sessions_for_user {#max-sessions-per-user} + +Maximum number of simultaneous sessions per authenticated user to the ClickHouse server. + +Example: + +``` xml + + + 1 + + + 2 + + + 0 + + + + + + single_session_user + + + + two_sessions_profile + + + + unlimited_sessions_profile + + +``` + +Default value: 0 (Infinite count of simultaneous sessions). diff --git a/docs/en/operations/settings/settings-profiles.md b/docs/en/operations/settings/settings-profiles.md index 2f39a75453c..d08266b8ef8 100644 --- a/docs/en/operations/settings/settings-profiles.md +++ b/docs/en/operations/settings/settings-profiles.md @@ -39,7 +39,7 @@ Example: 8
- + 1000000000 100000000000 @@ -67,6 +67,8 @@ Example: 50 100 + 4 + 1 diff --git a/docs/ru/operations/settings/query-complexity.md b/docs/ru/operations/settings/query-complexity.md index de9bb969085..5f3f4b74f51 100644 --- a/docs/ru/operations/settings/query-complexity.md +++ b/docs/ru/operations/settings/query-complexity.md @@ -314,3 +314,40 @@ FORMAT Null; При вставке данных, ClickHouse вычисляет количество партиций во вставленном блоке. Если число партиций больше, чем `max_partitions_per_insert_block`, ClickHouse генерирует исключение со следующим текстом: > «Too many partitions for single INSERT block (more than» + toString(max_parts) + «). The limit is controlled by ‘max_partitions_per_insert_block’ setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).» + +## max_sessions_for_user {#max-sessions-per-user} + +Максимальное количество одновременных сессий на одного аутентифицированного пользователя. + +Пример: + +``` xml + + + 1 + + + 2 + + + 0 + + + + + + single_session_profile + + + + two_sessions_profile + + + + unlimited_sessions_profile + + +``` + +Значение по умолчанию: 0 (неограниченное количество сессий). diff --git a/docs/ru/operations/settings/settings-profiles.md b/docs/ru/operations/settings/settings-profiles.md index ba2cb9a601f..0d094c637ac 100644 --- a/docs/ru/operations/settings/settings-profiles.md +++ b/docs/ru/operations/settings/settings-profiles.md @@ -39,7 +39,7 @@ SET profile = 'web' 8 - + 1000000000 100000000000 @@ -67,6 +67,7 @@ SET profile = 'web' 50 100 + 4 1 diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 9e9d8644539..e277c49e39d 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -328,9 +328,6 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & enabled_row_policies = access_control->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); - enabled_quota = access_control->getEnabledQuota( - *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); - enabled_settings = access_control->getEnabledSettings( *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); @@ -416,19 +413,32 @@ RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, co std::shared_ptr ContextAccess::getQuota() const { std::lock_guard lock{mutex}; - if (enabled_quota) - return enabled_quota; - static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota(); - return unlimited_quota; + + if (!enabled_quota) + { + if (roles_info) + { + enabled_quota = access_control->getEnabledQuota(*params.user_id, + user_name, + roles_info->enabled_roles, + params.address, + params.forwarded_address, + params.quota_key); + } + else + { + static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota(); + return unlimited_quota; + } + } + + return enabled_quota; } std::optional ContextAccess::getQuotaUsage() const { - std::lock_guard lock{mutex}; - if (enabled_quota) - return enabled_quota->getUsage(); - return {}; + return getQuota()->getUsage(); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 12f584cab83..74c6bbe0353 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -6,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -20,6 +22,39 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } +namespace +{ +struct SettingSourceRestrictions +{ + constexpr SettingSourceRestrictions() { allowed_sources.set(); } + + constexpr SettingSourceRestrictions(std::initializer_list allowed_sources_) + { + for (auto allowed_source : allowed_sources_) + setSourceAllowed(allowed_source, true); + } + + constexpr bool isSourceAllowed(SettingSource source) { return allowed_sources[source]; } + constexpr void setSourceAllowed(SettingSource source, bool allowed) { allowed_sources[source] = allowed; } + + std::bitset allowed_sources; +}; + +const std::unordered_map SETTINGS_SOURCE_RESTRICTIONS = { + {"max_sessions_for_user", {SettingSource::PROFILE}}, +}; + +SettingSourceRestrictions getSettingSourceRestrictions(std::string_view name) +{ + auto settingConstraintIter = SETTINGS_SOURCE_RESTRICTIONS.find(name); + if (settingConstraintIter != SETTINGS_SOURCE_RESTRICTIONS.end()) + return settingConstraintIter->second; + else + return SettingSourceRestrictions(); // allows everything +} + +} + SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_) { } @@ -98,7 +133,7 @@ void SettingsConstraints::merge(const SettingsConstraints & other) } -void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const +void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const { for (const auto & element : profile_elements) { @@ -108,19 +143,19 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting if (element.value) { SettingChange value(element.setting_name, *element.value); - check(current_settings, value); + check(current_settings, value, source); } if (element.min_value) { SettingChange value(element.setting_name, *element.min_value); - check(current_settings, value); + check(current_settings, value, source); } if (element.max_value) { SettingChange value(element.setting_name, *element.max_value); - check(current_settings, value); + check(current_settings, value, source); } SettingConstraintWritability new_value = SettingConstraintWritability::WRITABLE; @@ -142,24 +177,24 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting } } -void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change) const +void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change, SettingSource source) const { - checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); + checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION, source); } -void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes) const +void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const { for (const auto & change : changes) - check(current_settings, change); + check(current_settings, change, source); } -void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes) const +void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const { boost::range::remove_erase_if( changes, [&](SettingChange & change) -> bool { - return !checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); + return !checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION, source); }); } @@ -174,13 +209,13 @@ void SettingsConstraints::check(const MergeTreeSettings & current_settings, cons check(current_settings, change); } -void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const +void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const { boost::range::remove_erase_if( changes, [&](SettingChange & change) -> bool { - return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION); + return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION, source); }); } @@ -215,7 +250,10 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel return true; } -bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const +bool SettingsConstraints::checkImpl(const Settings & current_settings, + SettingChange & change, + ReactionOnViolation reaction, + SettingSource source) const { std::string_view setting_name = Settings::Traits::resolveName(change.name); @@ -247,7 +285,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - return getChecker(current_settings, setting_name).check(change, new_value, reaction); + return getChecker(current_settings, setting_name).check(change, new_value, reaction, source); } bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const @@ -255,10 +293,13 @@ bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, Field new_value; if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - return getMergeTreeChecker(change.name).check(change, new_value, reaction); + return getMergeTreeChecker(change.name).check(change, new_value, reaction, SettingSource::QUERY); } -bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const +bool SettingsConstraints::Checker::check(SettingChange & change, + const Field & new_value, + ReactionOnViolation reaction, + SettingSource source) const { if (!explain.empty()) { @@ -326,6 +367,14 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n change.value = max_value; } + if (!getSettingSourceRestrictions(setting_name).isSourceAllowed(source)) + { + if (reaction == THROW_ON_VIOLATION) + throw Exception(ErrorCodes::READONLY, "Setting {} is not allowed to be set by {}", setting_name, toString(source)); + else + return false; + } + return true; } diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 667d57a90ae..d09e60cc9d5 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace Poco::Util @@ -73,17 +74,18 @@ public: void merge(const SettingsConstraints & other); /// Checks whether `change` violates these constraints and throws an exception if so. - void check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const; - void check(const Settings & current_settings, const SettingChange & change) const; - void check(const Settings & current_settings, const SettingsChanges & changes) const; - void check(const Settings & current_settings, SettingsChanges & changes) const; + void check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const; + void check(const Settings & current_settings, const SettingChange & change, SettingSource source) const; + void check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const; + void check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const; /// Checks whether `change` violates these constraints and throws an exception if so. (setting short name is expected inside `changes`) void check(const MergeTreeSettings & current_settings, const SettingChange & change) const; void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const; /// Checks whether `change` violates these and clamps the `change` if so. - void clamp(const Settings & current_settings, SettingsChanges & changes) const; + void clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const; + friend bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right); friend bool operator !=(const SettingsConstraints & left, const SettingsConstraints & right) { return !(left == right); } @@ -133,7 +135,10 @@ private: {} // Perform checking - bool check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const; + bool check(SettingChange & change, + const Field & new_value, + ReactionOnViolation reaction, + SettingSource source) const; }; struct StringHash @@ -145,7 +150,11 @@ private: } }; - bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; + bool checkImpl(const Settings & current_settings, + SettingChange & change, + ReactionOnViolation reaction, + SettingSource source) const; + bool checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; Checker getChecker(const Settings & current_settings, std::string_view setting_name) const; diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 1723f85dc16..00e0ebd8b91 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -124,6 +124,9 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) continue; + /// Client can successfully connect to the server and + /// get ErrorCodes::USER_SESSION_LIMIT_EXCEEDED for suggestion connection. + /// We should not use std::cerr here, because this method works concurrently with the main thread. /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ae8d5f8796d..393486f805c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -582,6 +582,7 @@ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ M(699, INVALID_REDIS_TABLE_STRUCTURE) \ + M(700, USER_SESSION_LIMIT_EXCEEDED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/SettingSource.h b/src/Common/SettingSource.h new file mode 100644 index 00000000000..38e1bcae295 --- /dev/null +++ b/src/Common/SettingSource.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +namespace DB +{ + enum SettingSource + { + /// Query or session change: + /// SET = + /// SELECT ... SETTINGS [ = = = = ] + /// ALTER ROLE ... SETTINGS [ = = ] + /// ALTER USER ... SETTINGS [ = checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::ROLE); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..8a79bab0b0d 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -54,7 +54,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() settings_from_query = SettingsProfileElements{*query.settings, access_control}; if (!query.attach) - getContext()->checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::PROFILE); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..475ee270506 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -133,7 +133,7 @@ BlockIO InterpreterCreateUserQuery::execute() settings_from_query = SettingsProfileElements{*query.settings, access_control}; if (!query.attach) - getContext()->checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::USER); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d1b88988e5e..b10f85a2197 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -158,6 +159,7 @@ namespace CurrentMetrics extern const Metric IOWriterThreadsActive; } + namespace DB { @@ -276,6 +278,7 @@ struct ContextSharedPart : boost::noncopyable mutable QueryCachePtr query_cache; /// Cache of query results. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. ProcessList process_list; /// Executing queries at the moment. + SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) @@ -739,6 +742,9 @@ std::unique_lock Context::getLock() const ProcessList & Context::getProcessList() { return shared->process_list; } const ProcessList & Context::getProcessList() const { return shared->process_list; } OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } + +SessionTracker & Context::getSessionTracker() { return shared->session_tracker; } + MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } @@ -1094,7 +1100,7 @@ void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool se std::optional params; { auto lock = getLock(); - params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info}); + params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info }); } /// `temp_access` is used here only to extract information about the user, not to actually check access. /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. @@ -1157,13 +1163,6 @@ std::optional Context::getUserID() const } -void Context::setQuotaKey(String quota_key_) -{ - auto lock = getLock(); - client_info.quota_key = std::move(quota_key_); -} - - void Context::setCurrentRoles(const std::vector & current_roles_) { auto lock = getLock(); @@ -1303,7 +1302,7 @@ void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, boo { auto lock = getLock(); if (check_constraints) - checkSettingsConstraints(profiles_info.settings); + checkSettingsConstraints(profiles_info.settings, SettingSource::PROFILE); applySettingsChanges(profiles_info.settings); settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); } @@ -1857,29 +1856,29 @@ void Context::applySettingsChanges(const SettingsChanges & changes) } -void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements) const +void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements, source); } -void Context::checkSettingsConstraints(const SettingChange & change) const +void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change, source); } -void Context::checkSettingsConstraints(const SettingsChanges & changes) const +void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source); } -void Context::checkSettingsConstraints(SettingsChanges & changes) const +void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source); } -void Context::clampToSettingsConstraints(SettingsChanges & changes) const +void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes, source); } void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0d567816ec9..676eb8412e5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -202,6 +203,8 @@ using MergeTreeMetadataCachePtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; +class SessionTracker; + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -539,8 +542,6 @@ public: String getUserName() const; - void setQuotaKey(String quota_key_); - void setCurrentRoles(const std::vector & current_roles_); void setCurrentRolesDefault(); boost::container::flat_set getCurrentRoles() const; @@ -735,11 +736,11 @@ public: void applySettingsChanges(const SettingsChanges & changes); /// Checks the constraints. - void checkSettingsConstraints(const SettingsProfileElements & profile_elements) const; - void checkSettingsConstraints(const SettingChange & change) const; - void checkSettingsConstraints(const SettingsChanges & changes) const; - void checkSettingsConstraints(SettingsChanges & changes) const; - void clampToSettingsConstraints(SettingsChanges & changes) const; + void checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const; + void checkSettingsConstraints(const SettingChange & change, SettingSource source) const; + void checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const; + void checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const; + void clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const; void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const; /// Reset settings to default value @@ -861,6 +862,8 @@ public: OvercommitTracker * getGlobalOvercommitTracker() const; + SessionTracker & getSessionTracker(); + MergeList & getMergeList(); const MergeList & getMergeList() const; diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index e9118b747e5..2c0baa0d4b3 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -15,7 +15,7 @@ namespace DB BlockIO InterpreterSetQuery::execute() { const auto & ast = query_ptr->as(); - getContext()->checkSettingsConstraints(ast.changes); + getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY); auto session_context = getContext()->getSessionContext(); session_context->applySettingsChanges(ast.changes); session_context->addQueryParameters(ast.query_parameters); @@ -28,7 +28,7 @@ void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constrain { const auto & ast = query_ptr->as(); if (!ignore_setting_constraints) - getContext()->checkSettingsConstraints(ast.changes); + getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY); getContext()->applySettingsChanges(ast.changes); getContext()->resetSettingsToDefaultValue(ast.default_settings); } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index cadf619700c..de2a779b740 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -3,11 +3,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -200,7 +202,6 @@ private: LOG_TEST(log, "Schedule closing session with session_id: {}, user_id: {}", session.key.second, session.key.first); - } void cleanThread() @@ -336,6 +337,9 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So if (session_context) throw Exception(ErrorCodes::LOGICAL_ERROR, "If there is a session context it must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before authentication finish"); + auto address = address_; if ((address == Poco::Net::SocketAddress{}) && (prepared_client_info->interface == ClientInfo::Interface::LOCAL)) address = Poco::Net::SocketAddress{"127.0.0.1", 0}; @@ -490,6 +494,8 @@ ContextMutablePtr Session::makeSessionContext() throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context"); if (!user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); LOG_DEBUG(log, "{} Creating session context with user_id: {}", toString(auth_id), toString(*user_id)); @@ -503,13 +509,17 @@ ContextMutablePtr Session::makeSessionContext() prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. - if (user_id) - new_session_context->setUser(*user_id); + new_session_context->setUser(*user_id); /// Session context is ready. session_context = new_session_context; user = session_context->getUser(); + session_tracker_handle = session_context->getSessionTracker().trackSession( + *user_id, + {}, + session_context->getSettingsRef().max_sessions_for_user); + return session_context; } @@ -521,6 +531,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context"); if (!user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}", toString(auth_id), session_name_, toString(*user_id)); @@ -541,9 +553,23 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); + auto access = new_session_context->getAccess(); + UInt64 max_sessions_for_user = 0; /// Set user information for the new context: current profiles, roles, access rights. - if (user_id && !new_session_context->getAccess()->tryGetUser()) + if (!access->tryGetUser()) + { new_session_context->setUser(*user_id); + max_sessions_for_user = new_session_context->getSettingsRef().max_sessions_for_user; + } + else + { + // Always get setting from profile + // profile can be changed by ALTER PROFILE during single session + auto settings = access->getDefaultSettings(); + const Field * max_session_for_user_field = settings.tryGet("max_sessions_for_user"); + if (max_session_for_user_field) + max_sessions_for_user = max_session_for_user_field->safeGet(); + } /// Session context is ready. session_context = std::move(new_session_context); @@ -551,6 +577,11 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: named_session_created = new_named_session_created; user = session_context->getUser(); + session_tracker_handle = session_context->getSessionTracker().trackSession( + *user_id, + { session_name_ }, + max_sessions_for_user); + return session_context; } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 36f811ccd24..51c0e3c71fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -113,6 +114,8 @@ private: std::shared_ptr named_session; bool named_session_created = false; + SessionTracker::SessionTrackerHandle session_tracker_handle; + Poco::Logger * log = nullptr; }; diff --git a/src/Interpreters/SessionTracker.cpp b/src/Interpreters/SessionTracker.cpp new file mode 100644 index 00000000000..4636766e288 --- /dev/null +++ b/src/Interpreters/SessionTracker.cpp @@ -0,0 +1,62 @@ +#include "SessionTracker.h" + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int USER_SESSION_LIMIT_EXCEEDED; +} + +SessionTracker::Session::Session(SessionTracker & tracker_, + const UUID& user_id_, + SessionInfos::const_iterator session_info_iter_) noexcept + : tracker(tracker_), user_id(user_id_), session_info_iter(session_info_iter_) +{ +} + +SessionTracker::Session::~Session() +{ + tracker.stopTracking(user_id, session_info_iter); +} + +SessionTracker::SessionTrackerHandle +SessionTracker::trackSession(const UUID & user_id, + const SessionInfo & session_info, + size_t max_sessions_for_user) +{ + std::lock_guard lock(mutex); + + auto sessions_for_user_iter = sessions_for_user.find(user_id); + if (sessions_for_user_iter == sessions_for_user.end()) + sessions_for_user_iter = sessions_for_user.emplace(user_id, SessionInfos()).first; + + SessionInfos & session_infos = sessions_for_user_iter->second; + if (max_sessions_for_user && session_infos.size() >= max_sessions_for_user) + { + throw Exception(ErrorCodes::USER_SESSION_LIMIT_EXCEEDED, + "User {} has overflown session count {}", + toString(user_id), + max_sessions_for_user); + } + + session_infos.emplace_front(session_info); + + return std::make_unique(*this, user_id, session_infos.begin()); +} + +void SessionTracker::stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter) +{ + std::lock_guard lock(mutex); + + auto sessions_for_user_iter = sessions_for_user.find(user_id); + chassert(sessions_for_user_iter != sessions_for_user.end()); + + sessions_for_user_iter->second.erase(session_info_iter); + if (sessions_for_user_iter->second.empty()) + sessions_for_user.erase(sessions_for_user_iter); +} + +} diff --git a/src/Interpreters/SessionTracker.h b/src/Interpreters/SessionTracker.h new file mode 100644 index 00000000000..0827213aeed --- /dev/null +++ b/src/Interpreters/SessionTracker.h @@ -0,0 +1,60 @@ +#pragma once + +#include "ClientInfo.h" + +#include +#include +#include +#include + +namespace DB +{ + +struct SessionInfo +{ + const String session_id; +}; + +using SessionInfos = std::list; + +using SessionsForUser = std::unordered_map; + +class SessionTracker; + +class SessionTracker +{ +public: + class Session : boost::noncopyable + { + public: + explicit Session(SessionTracker & tracker_, + const UUID & user_id_, + SessionInfos::const_iterator session_info_iter_) noexcept; + + ~Session(); + + private: + friend class SessionTracker; + + SessionTracker & tracker; + const UUID user_id; + const SessionInfos::const_iterator session_info_iter; + }; + + using SessionTrackerHandle = std::unique_ptr; + + SessionTrackerHandle trackSession(const UUID & user_id, + const SessionInfo & session_info, + size_t max_sessions_for_user); + +private: + /// disallow manual messing with session tracking + friend class Session; + + std::mutex mutex; + SessionsForUser sessions_for_user TSA_GUARDED_BY(mutex); + + void stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter); +}; + +} diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 67d30012b0e..3370a8c009b 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -833,7 +833,7 @@ namespace { settings_changes.push_back({key, value}); } - query_context->checkSettingsConstraints(settings_changes); + query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); query_context->applySettingsChanges(settings_changes); query_context->setCurrentQueryId(query_info.query_id()); @@ -1118,7 +1118,7 @@ namespace SettingsChanges settings_changes; for (const auto & [key, value] : external_table.settings()) settings_changes.push_back({key, value}); - external_table_context->checkSettingsConstraints(settings_changes); + external_table_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); external_table_context->applySettingsChanges(settings_changes); } auto in = external_table_context->getInputFormat( diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index a0bfcd49dfd..ff5690a3b07 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -764,7 +764,7 @@ void HTTPHandler::processQuery( context->setDefaultFormat(default_format); /// For external data we also want settings - context->checkSettingsConstraints(settings_changes); + context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); context->applySettingsChanges(settings_changes); /// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a747f06f1ce..5f3a7614eee 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -184,14 +184,17 @@ void TCPHandler::runImpl() try { receiveHello(); + + /// In interserver mode queries are executed without a session context. + if (!is_interserver_mode) + session->makeSessionContext(); + sendHello(); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); - if (!is_interserver_mode) /// In interserver mode queries are executed without a session context. + if (!is_interserver_mode) { - session->makeSessionContext(); - /// If session created, then settings in session context has been updated. /// So it's better to update the connection settings for flexibility. extractConnectionSettingsFromContext(session->sessionContext()); @@ -1181,7 +1184,6 @@ std::unique_ptr TCPHandler::makeSession() res->setClientName(client_name); res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); - res->setQuotaClientKey(quota_key); res->setClientInterface(interface); return res; @@ -1274,11 +1276,10 @@ void TCPHandler::receiveHello() void TCPHandler::receiveAddendum() { if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY) - { readStringBinary(quota_key, *in); - if (!is_interserver_mode) - session->setQuotaClientKey(quota_key); - } + + if (!is_interserver_mode) + session->setQuotaClientKey(quota_key); } @@ -1591,12 +1592,12 @@ void TCPHandler::receiveQuery() if (query_kind == ClientInfo::QueryKind::INITIAL_QUERY) { /// Throw an exception if the passed settings violate the constraints. - query_context->checkSettingsConstraints(settings_changes); + query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); } else { /// Quietly clamp to the constraints if it's not an initial query. - query_context->clampToSettingsConstraints(settings_changes); + query_context->clampToSettingsConstraints(settings_changes, SettingSource::QUERY); } query_context->applySettingsChanges(settings_changes); diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 6e1604f4eb5..cb2b98937ca 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -81,5 +81,15 @@ "test_system_flush_logs/test.py::test_log_buffer_size_rows_flush_threshold", "test_system_flush_logs/test.py::test_log_max_size", "test_crash_log/test.py::test_pkill_query_log", - "test_crash_log/test.py::test_pkill" + "test_crash_log/test.py::test_pkill", + + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_postgres", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_mysql", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_end_session" ] diff --git a/tests/integration/test_profile_max_sessions_for_user/__init__.py b/tests/integration/test_profile_max_sessions_for_user/__init__.py new file mode 100755 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem b/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem new file mode 100755 index 00000000000..2e6cee0798d --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem @@ -0,0 +1,8 @@ +-----BEGIN DH PARAMETERS----- +MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR +XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5 +Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb +9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf +slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb +dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg== +-----END DH PARAMETERS----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/log.xml b/tests/integration/test_profile_max_sessions_for_user/configs/log.xml new file mode 100644 index 00000000000..22f95a8bd5d --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml b/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml new file mode 100644 index 00000000000..3123c4a3d9c --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/server.crt b/tests/integration/test_profile_max_sessions_for_user/configs/server.crt new file mode 100755 index 00000000000..070d37f3b77 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/server.crt @@ -0,0 +1,18 @@ +-----BEGIN CERTIFICATE----- +MIIC+zCCAeOgAwIBAgIJANhP897Se2gmMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAeFw0yMDA0MTgyMTE2NDBaFw0yMTA0MTgyMTE2NDBaMBQx +EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC +ggEBAM92kcojQoMsjZ9YGhPMY6h/fDUsZeSKHLxgqE6wbmfU1oZKCPWqnvl+4n0J +pnT5h1ETxxYZLepimKq0DEVPUTmCl0xmcKbtUNiaTUKYKsdita6b2vZCX9wUPN9p +2Kjnm41l+aZNqIEBhIgHNWg9qowi20y0EIXR79jQLwwaInHAaJLZxVsqY2zjQ/D7 +1Zh82MXud7iqxBQiEfw9Cz35UFA239R8QTlPkVQfsN1gfLxnLk24QUX3o+hbUI1g +nlSpyYDHYQlOmwz8doDs6THHAZNJ4bPE9xHNFpw6dGZdbtH+IKQ/qRZIiOaiNuzJ +IOHl6XQDRDkW2LMTiCQ6fjC7Pz8CAwEAAaNQME4wHQYDVR0OBBYEFFvhaA/Eguyf +BXkMj8BkNLBqMnz2MB8GA1UdIwQYMBaAFFvhaA/EguyfBXkMj8BkNLBqMnz2MAwG +A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBACeU/oL48eVAKH7NQntHhRaJ +ZGeQzKIjrSBjFo8BGXD1nJZhUeFsylLrhCkC8/5/3grE3BNVX9bxcGjO81C9Mn4U +t0z13d6ovJjCZSQArtLwgeJGlpH7gNdD3DyT8DQmrqYVnmnB7UmBu45XH1LWGQZr +FAOhGRVs6s6mNj8QlLMgdmsOeOQnsGCMdoss8zV9vO2dc4A5SDSSL2mqGGY4Yjtt +X+XlEhXXnksGyx8NGVOZX4wcj8WeCAj/lihQ7Zh6XYwZH9i+E46ompUwoziZnNPu +2RH63tLNCxkOY2HF5VMlbMmzer3FkhlM6TAZZRPcvSphKPwXK4A33yqc6wnWvpc= +-----END CERTIFICATE----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/server.key b/tests/integration/test_profile_max_sessions_for_user/configs/server.key new file mode 100755 index 00000000000..b3dee82dcda --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDPdpHKI0KDLI2f +WBoTzGOof3w1LGXkihy8YKhOsG5n1NaGSgj1qp75fuJ9CaZ0+YdRE8cWGS3qYpiq +tAxFT1E5gpdMZnCm7VDYmk1CmCrHYrWum9r2Ql/cFDzfadio55uNZfmmTaiBAYSI +BzVoPaqMIttMtBCF0e/Y0C8MGiJxwGiS2cVbKmNs40Pw+9WYfNjF7ne4qsQUIhH8 +PQs9+VBQNt/UfEE5T5FUH7DdYHy8Zy5NuEFF96PoW1CNYJ5UqcmAx2EJTpsM/HaA +7OkxxwGTSeGzxPcRzRacOnRmXW7R/iCkP6kWSIjmojbsySDh5el0A0Q5FtizE4gk +On4wuz8/AgMBAAECggEAJ54J2yL+mZQRe2NUn4FBarTloDXZQ1pIgISov1Ybz0Iq +sTxEF728XAKp95y3J9Fa0NXJB+RJC2BGrRpy2W17IlNY1yMc0hOxg5t7s4LhcG/e +J/jlSG+GZL2MnlFVKXQJFWhq0yIzUmdayqstvLlB7z7cx/n+yb88YRfoVBRNjZEL +Tdrsw+087igDjrIxZJ3eMN5Wi434n9s4yAoRQC1bP5wcWx0gD4MzdmL8ip6suiRc +LRuBAhV/Op812xlxUhrF5dInUM9OLlGTXpUzexAS8Cyy7S4bfkW2BaCxTF7I7TFw +Whx28CKn/G49tIuU0m6AlxWbXpLVePTFyMb7RJz5cQKBgQD7VQd2u3HM6eE3PcXD +p6ObdLTUk8OAJ5BMmADFc71W0Epyo26/e8KXKGYGxE2W3fr13y+9b0fl5fxZPuhS +MgvXEO7rItAVsLcp0IzaqY0WUee2b4XWPAU0XuPqvjYMpx8H5OEHqFK6lhZysAqM +X7Ot3/Hux9X0MC4v5a/HNbDUOQKBgQDTUPaP3ADRrmpmE2sWuzWEnCSEz5f0tCLO +wTqhV/UraWUNlAbgK5NB790IjH/gotBSqqNPLJwJh0LUfClKM4LiaHsEag0OArOF +GhPMK1Ohps8c2RRsiG8+hxX2HEHeAVbkouEDPDiHdIW/92pBViDoETXL6qxDKbm9 +LkOcVeDfNwKBgQChh1xsqrvQ/t+IKWNZA/zahH9TwEP9sW/ESkz0mhYuHWA7nV4o +ItpFW+l2n+Nd+vy32OFN1p9W2iD9GrklWpTRfEiRRqaFyjVt4mMkhaPvnGRXlAVo +Utrldbb1v5ntN9txr2ARE9VXpe53dzzQSxGnxi4vUK/paK3GitAWMCOdwQKBgQCi +hmGsUXQb0P6qVYMGr6PAw2re7t8baLRguoMCdqjs45nCMLh9D2apzvb8TTtJJU/+ +VJlYGqJEPdDrpjcHh8jBo8QBqCM0RGWYGG9jl2syKB6hPGCV/PU6bSE58Y/DVNpk +7NUM7PM5UyhPddY2PC0A78Ole29UFLJzSzLa+b4DTwKBgH9Wh2k4YPnPcRrX89UL +eSwWa1CGq6HWX8Kd5qyz256aeHWuG5nv15+rBt+D7nwajUsqeVkAXz5H/dHuG1xz +jb7RW+pEjx0GVAmIbkM9vOLqEUfHHHPuk4AXCGGZ5sarPiKg4BHKBBsY1dpoO5UH +0j71fRA6zurHnTXDaCLWlUpZ +-----END PRIVATE KEY----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml b/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml new file mode 100644 index 00000000000..778d327c460 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml @@ -0,0 +1,17 @@ + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/users.xml b/tests/integration/test_profile_max_sessions_for_user/configs/users.xml new file mode 100644 index 00000000000..3bed673b2ca --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/users.xml @@ -0,0 +1,16 @@ + + + + 2 + 0 + + + + + + + + 123 + + + diff --git a/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto b/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py new file mode 100755 index 00000000000..9e6a10e7e15 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -0,0 +1,222 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +MAX_SESSIONS_FOR_USER = 2 +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 + +TEST_USER = "test_user" +TEST_PASSWORD = "123" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ + {proto_dir}/clickhouse_grpc.proto".format( + proto_dir=proto_dir, gen_dir=gen_dir + ), + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/ssl_conf.xml", + "configs/dhparam.pem", + "configs/server.crt", + "configs/server.key", + ], + user_configs=["configs/users.xml"], + env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"}, +) + + +def get_query(name, id): + return f"SElECT '{name}', {id}, sleep(1)" + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +def grpc_query(query_text, channel, session_id_): + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query_text, + session_id=session_id_, + user_name=TEST_USER, + password=TEST_PASSWORD, + ) + + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + return result.output.decode(DEFAULT_ENCODING) + + +def threaded_run_test(sessions): + thread_list = [] + for i in range(len(sessions)): + thread = ThreadWithException(target=sessions[i], args=(i,)) + thread_list.append(thread) + thread.start() + + for thread in thread_list: + thread.join() + + exception_count = 0 + for i in range(len(sessions)): + if thread_list[i].run_exception != None: + exception_count += 1 + + assert exception_count == 1 + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +class ThreadWithException(threading.Thread): + run_exception = None + + def run(self): + try: + super().run() + except: + self.run_exception = sys.exc_info() + + def join(self): + super().join() + + +def postgres_session(id): + ch = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=TEST_USER, + password=TEST_PASSWORD, + database="default", + ) + cur = ch.cursor() + cur.execute(get_query("postgres_session", id)) + cur.fetchall() + + +def mysql_session(id): + client = pymysql.connections.Connection( + host=instance.ip_address, + user=TEST_USER, + password=TEST_PASSWORD, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + cursor.execute(get_query("mysql_session", id)) + cursor.fetchall() + + +def tcp_session(id): + instance.query(get_query("tcp_session", id), user=TEST_USER, password=TEST_PASSWORD) + + +def http_session(id): + instance.http_query( + get_query("http_session", id), user=TEST_USER, password=TEST_PASSWORD + ) + + +def http_named_session(id): + instance.http_query( + get_query("http_named_session", id), + user=TEST_USER, + password=TEST_PASSWORD, + params={"session_id": id}, + ) + + +def grpc_session(id): + grpc_query( + get_query("grpc_session", id), grpc_create_insecure_channel(), f"session_{id}" + ) + + +def test_profile_max_sessions_for_user_tcp(started_cluster): + threaded_run_test([tcp_session] * 3) + + +def test_profile_max_sessions_for_user_postgres(started_cluster): + threaded_run_test([postgres_session] * 3) + + +def test_profile_max_sessions_for_user_mysql(started_cluster): + threaded_run_test([mysql_session] * 3) + + +def test_profile_max_sessions_for_user_http(started_cluster): + threaded_run_test([http_session] * 3) + + +def test_profile_max_sessions_for_user_http_named_session(started_cluster): + threaded_run_test([http_named_session] * 3) + + +def test_profile_max_sessions_for_user_grpc(started_cluster): + threaded_run_test([grpc_session] * 3) + + +def test_profile_max_sessions_for_user_tcp_and_others(started_cluster): + threaded_run_test([tcp_session, grpc_session, grpc_session]) + threaded_run_test([tcp_session, http_session, http_session]) + threaded_run_test([tcp_session, mysql_session, mysql_session]) + threaded_run_test([tcp_session, postgres_session, postgres_session]) + threaded_run_test([tcp_session, http_session, postgres_session]) + threaded_run_test([tcp_session, postgres_session, http_session]) + + +def test_profile_max_sessions_for_user_end_session(started_cluster): + for conection_func in [ + tcp_session, + http_session, + grpc_session, + mysql_session, + postgres_session, + ]: + threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) + threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) + + +def test_profile_max_sessions_for_user_end_session(started_cluster): + instance.query_and_get_error("SET max_sessions_for_user = 10") diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference new file mode 100644 index 00000000000..f80f8738ff8 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference @@ -0,0 +1,12 @@ +test_alter_profile case: max_session_count 1 alter_sessions_count 1 +test_alter_profile case: max_session_count 2 alter_sessions_count 1 +USER_SESSION_LIMIT_EXCEEDED +test_alter_profile case: max_session_count 1 alter_sessions_count 2 +test_alter_profile case: max_session_count 2 alter_sessions_count 2 +READONLY +READONLY +READONLY +READONLY +READONLY +READONLY +READONLY diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh new file mode 100755 index 00000000000..546c54a4de9 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SESSION_ID_PREFIX="02832_alter_max_sessions_session_$$" +PROFILE="02832_alter_max_sessions_profile_$$" +USER="02832_alter_max_sessions_user_$$" +USER2="02832_alter_max_sessions_user_two_$$" +ROLE="02832_alter_max_sessions_role_$$" + +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'" +${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"CREATE SETTINGS PROFILE ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"CREATE USER '${USER}' SETTINGS PROFILE '${PROFILE}'" + +function test_alter_profile() +{ + local max_session_count="$1" + local alter_sessions_count="$2" + echo $"test_alter_profile case: max_session_count ${max_session_count} alter_sessions_count ${alter_sessions_count}" + + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}" + + # Create sesssions with $max_session_count resriction + for ((i = 1 ; i <= ${max_session_count} ; i++)); do + local session_id="${SESSION_ID_PREFIX}_${i}" + # Skip output from this query + ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=0" --data-binary "SELECT 1" > /dev/null + done + + # Update resriction to $alter_sessions_count + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}" + + # Simultaneous sessions should use max settings from profile ($alter_sessions_count) + for ((i = 1 ; i <= ${max_session_count} ; i++)); do + local session_id="${SESSION_ID_PREFIX}_${i}" + # ignore select 1, we need only errors + ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=1" --data-binary "select sleep(0.3)" | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' & + done + + wait +} + +test_alter_profile 1 1 +test_alter_profile 2 1 +test_alter_profile 1 2 +test_alter_profile 2 2 + +${CLICKHOUSE_CLIENT} -q "SELECT 1 SETTINGS max_sessions_for_user = 1" 2>&1 | grep -m 1 -o 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"SET max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} --max_sessions_for_user=1 -q $"SELECT 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +# max_sessions_for_user is profile setting +${CLICKHOUSE_CLIENT} -q $"CREATE USER ${USER2} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"ALTER USER ${USER} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE}" +${CLICKHOUSE_CLIENT} -q $"ALTER ROLE ${ROLE} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 + +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'" +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER2}'" +${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"DROP ROLE IF EXISTS ${ROLE}" From afb76101b797a144bc2e8d6cff5109c9036fc46a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 05:23:28 +0200 Subject: [PATCH 859/871] Remove obsolete part of a check name --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- tests/ci/ci_config.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e5b797beebd..ae1862e327f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3643,7 +3643,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang) + CHECK_NAME=Unit tests (release) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..d97b9975c3c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -4541,7 +4541,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang) + CHECK_NAME=Unit tests (release) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bea654ca76f..517e40fd2d6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -346,7 +346,7 @@ CI_CONFIG = { "Compatibility check (aarch64)": { "required_build": "package_aarch64", }, - "Unit tests (release-clang)": { + "Unit tests (release)": { "required_build": "binary_release", }, "Unit tests (asan)": { @@ -509,7 +509,7 @@ REQUIRED_CHECKS = [ "Style Check", "Unit tests (asan)", "Unit tests (msan)", - "Unit tests (release-clang)", + "Unit tests (release)", "Unit tests (tsan)", "Unit tests (ubsan)", ] From efad90d0f2731142677287495f4d8f47a4c32b51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 07:26:36 +0200 Subject: [PATCH 860/871] Maybe fix TLS tests --- tests/integration/test_ssl_cert_authentication/test.py | 6 +++--- tests/integration/test_tlsv1_3/test.py | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index ff2de7491e1..fe6eb52e50e 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -208,7 +208,7 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -314,7 +314,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -334,7 +334,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index f5c2be51ed7..094804bf963 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -96,7 +96,7 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -202,7 +202,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -222,7 +222,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" From 010f3f1db1fc8adad0a74f424e153c28f7072e16 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 31 Jul 2023 05:39:49 +0000 Subject: [PATCH 861/871] Automatic style fix --- .../integration/test_ssl_cert_authentication/test.py | 12 +++++++++--- tests/integration/test_tlsv1_3/test.py | 12 +++++++++--- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index fe6eb52e50e..d1ae39ca378 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -208,7 +208,9 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -314,7 +316,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -334,7 +338,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index 094804bf963..87c03c56f91 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -96,7 +96,9 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -202,7 +204,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -222,7 +226,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" From 2e0d82765c2dbb4e021c9f174eaba8637f74401f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 11:43:04 +0300 Subject: [PATCH 862/871] Move "reconfig" to experimental in the changelog See https://github.com/ClickHouse/ClickHouse/issues/52798 Automatic continuous fuzzing has found an issue. --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f401b346726..bf26708ebb0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,7 +23,6 @@ * Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). -* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). * Kafka connector can fetch Avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). * Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). * Add a column `is_obsolete` to `system.settings` and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). @@ -124,6 +123,7 @@ * (experimental MaterializedMySQL) Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). * Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). * The `session_timezone` setting (new in version 23.6) is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). It is suspected that this feature is incomplete. #### Build/Testing/Packaging Improvement * Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From a473dc4b51a62f8b44137a9b84c8dc09f6b97542 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 09:24:06 +0000 Subject: [PATCH 863/871] Wait for response --- src/Coordination/KeeperServer.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index a4c3d91e1c9..88b9f1cedb4 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -794,8 +794,14 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) std::lock_guard _{server_write_mutex}; if (const auto * add = std::get_if(&action)) - return raft_instance->get_srv_config(add->id) != nullptr - || raft_instance->add_srv(static_cast(*add))->get_accepted(); + { + if (raft_instance->get_srv_config(add->id) != nullptr) + return true; + + auto resp = raft_instance->add_srv(static_cast(*add)); + resp->get(); + return resp->get_accepted(); + } else if (const auto * remove = std::get_if(&action)) { if (remove->id == raft_instance->get_leader()) @@ -807,8 +813,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) return false; } - return raft_instance->get_srv_config(remove->id) == nullptr - || raft_instance->remove_srv(remove->id)->get_accepted(); + if (raft_instance->get_srv_config(remove->id) == nullptr) + return true; + + auto resp = raft_instance->remove_srv(remove->id); + resp->get(); + return resp->get_accepted(); } else if (const auto * update = std::get_if(&action)) { From 7c49105cd56395bde38f0d7aff862c65c78c6989 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 12:27:35 +0200 Subject: [PATCH 864/871] Allow OOM in Stress and Upgrade checks --- docker/test/stress/run.sh | 6 ++++++ docker/test/upgrade/run.sh | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9217fcfddd9..a2264b8f3e6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -233,4 +233,10 @@ rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv +# But OOMs in stress test are allowed +if rg 'OOM in dmesg|Signal 9' /test_output/check_status.tsv +then + sed -i 's/failure/success/' /test_output/check_status.tsv +fi + collect_core_dumps diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 13c352d5d41..d6cd6987e83 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -231,4 +231,10 @@ rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv +# But OOMs in stress test are allowed +if rg 'OOM in dmesg|Signal 9' /test_output/check_status.tsv +then + sed -i 's/failure/success/' /test_output/check_status.tsv +fi + collect_core_dumps From b683477b2034aaa3a5af7e21795d4a928cd1a719 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 14:14:15 +0200 Subject: [PATCH 865/871] Better dynamic disk configuration --- src/Disks/getDiskConfigurationFromAST.cpp | 10 ++++---- src/Disks/getDiskConfigurationFromAST.h | 8 +++---- src/Disks/getOrCreateDiskFromAST.cpp | 24 ++++++++----------- src/Parsers/ParserSetQuery.cpp | 4 ++-- ...2808_custom_disk_with_user_defined_name.sh | 4 ++-- 5 files changed, 22 insertions(+), 28 deletions(-) diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp index 4b1323b4db8..76a257d3b52 100644 --- a/src/Disks/getDiskConfigurationFromAST.cpp +++ b/src/Disks/getDiskConfigurationFromAST.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes message.empty() ? "" : ": " + message); } -Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +Poco::AutoPtr getDiskConfigurationFromASTImpl(const ASTs & disk_args, ContextPtr context) { if (disk_args.empty()) throwBadConfiguration("expected non-empty list of arguments"); @@ -39,8 +39,6 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st Poco::AutoPtr xml_document(new Poco::XML::Document()); Poco::AutoPtr root(xml_document->createElement("disk")); xml_document->appendChild(root); - Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); - root->appendChild(disk_configuration); for (const auto & arg : disk_args) { @@ -62,7 +60,7 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st const std::string & key = key_identifier->name(); Poco::AutoPtr key_element(xml_document->createElement(key)); - disk_configuration->appendChild(key_element); + root->appendChild(key_element); if (!function_args[1]->as() && !function_args[1]->as()) throwBadConfiguration("expected values to be literals or identifiers"); @@ -75,9 +73,9 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st return xml_document; } -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +DiskConfigurationPtr getDiskConfigurationFromAST(const ASTs & disk_args, ContextPtr context) { - auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); + auto xml_document = getDiskConfigurationFromASTImpl(disk_args, context); Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); conf->load(xml_document); return conf; diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h index 5697955e914..f23fb37b9dc 100644 --- a/src/Disks/getDiskConfigurationFromAST.h +++ b/src/Disks/getDiskConfigurationFromAST.h @@ -14,19 +14,19 @@ using DiskConfigurationPtr = Poco::AutoPtr; /** * Transform a list of pairs ( key1=value1, key2=value2, ... ), where keys and values are ASTLiteral or ASTIdentifier * into - * + * * value1 * value2 * ... - * + * * * Used in case disk configuration is passed via AST when creating * a disk object on-the-fly without any configuration file. */ -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); +DiskConfigurationPtr getDiskConfigurationFromAST(const ASTs & disk_args, ContextPtr context); /// The same as above function, but return XML::Document for easier modification of result configuration. -[[ maybe_unused ]] Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context); +[[ maybe_unused ]] Poco::AutoPtr getDiskConfigurationFromASTImpl(const ASTs & disk_args, ContextPtr context); /* * A reverse function. diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index a9a0e972bd1..da318303f62 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -26,8 +26,16 @@ namespace { std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(function_args, context); + std::string disk_name; - if (function.name == "disk") + if (config->has("name")) + { + disk_name = config->getString("name"); + } + else { /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk @@ -36,21 +44,9 @@ namespace disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); } - else - { - static constexpr std::string_view custom_disk_prefix = "disk_"; - - if (function.name.size() <= custom_disk_prefix.size() || !function.name.starts_with(custom_disk_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", function.name); - - disk_name = function.name.substr(custom_disk_prefix.size()); - } auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(disk_name, function_args, context); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + auto disk = DiskFactory::instance().create(disk_name, *config, "", context, disks_map); /// Mark that disk can be used without storage policy. disk->markDiskAsCustom(); return disk; diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 727d037112f..4df74c2dd82 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -215,7 +215,7 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); /// for SETTINGS disk=disk(type='s3', path='', ...) - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { tryGetIdentifierNameInto(name, change.name); change.value = createFieldFromAST(function_ast); @@ -280,7 +280,7 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault( node = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) node = std::make_shared(Field(static_cast(0))); - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { change.name = name; change.value = createFieldFromAST(function_ast); diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index a43cd6deb9e..333bc1bc25d 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_s3_disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ 2>&1 | grep -q "Disk with name \`s3_disk\` already exist" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" @@ -25,7 +25,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ $CLICKHOUSE_CLIENT -nm --query """ From 985b2a010af9ba5de9e72fc4623ea05e7d91c4b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 15:17:47 +0300 Subject: [PATCH 866/871] Add a tool to upload `-ftime-trace` to ClickHouse (#52776) * Add a tool to upload `-ftime-trace` to ClickHouse * Add a tool to upload `-ftime-trace` to ClickHouse * Add a tool to upload `-ftime-trace` to ClickHouse --- .../prepare-time-trace/prepare-time-trace.sh | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100755 utils/prepare-time-trace/prepare-time-trace.sh diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh new file mode 100755 index 00000000000..300a32b0fd1 --- /dev/null +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -0,0 +1,51 @@ +#!/bin/bash + +# This scripts transforms the output of clang's -ftime-trace JSON files into a format to upload to ClickHouse + +# Example: +# mkdir time_trace +# utils/prepare-time-trace/prepare-time-trace.sh build time_trace + +# See also https://docs.google.com/document/d/1CvAClvFfyA5R-PhYUmn5OOQtYMH4h6I0nSsKchNAySU/preview + +< \"${OUTPUT_DIR}/\$\$\" +" + +# Now you can upload it as follows: + +#cat "$OUTPUT_DIR"/* | clickhouse-client --progress --query "INSERT INTO build_time_trace (extra_column_names, file, library, time, pid, tid, ph, ts, dur, cat, name, detail, count, avgMs, args_name) FORMAT JSONCompactEachRow" From efba3a21139adf8004d975b2d276edfa39782b19 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Jul 2023 14:32:09 +0200 Subject: [PATCH 867/871] Add more logging and touch test for materialize mysql --- src/Common/mysqlxx/Pool.cpp | 3 +++ src/Common/mysqlxx/mysqlxx/Pool.h | 2 +- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 2 ++ src/Databases/MySQL/DatabaseMaterializedMySQL.h | 1 + src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 6 +++++- .../integration/test_materialized_mysql_database/test.py | 8 -------- 6 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 64a69c48e1d..43df0aa6708 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -153,7 +153,10 @@ Pool::Entry Pool::get(uint64_t wait_timeout) for (auto & connection : connections) { if (connection->ref_count == 0) + { + logger.test("Found free connection in pool, returning it to the caller"); return Entry(connection, this); + } } logger.trace("(%s): Trying to allocate a new connection.", getDescription()); diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index c2190fba684..52d116e39ce 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -26,7 +26,7 @@ namespace mysqlxx * * void thread() * { - * mysqlxx::Pool::Entry connection = pool.Get(); + * mysqlxx::Pool::Entry connection = pool.Get(); * std::string s = connection->query("SELECT 'Hello, world!' AS world").use().fetch()["world"].getString(); * } * TODO: simplify with PoolBase. diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 653c2dc27b6..f7e669d9feb 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -65,6 +65,7 @@ void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exceptio void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) { + LOG_TRACE(log, "Starting MaterializeMySQL tables"); DatabaseAtomic::startupTables(thread_pool, mode); if (mode < LoadingStrictnessLevel::FORCE_ATTACH) @@ -122,6 +123,7 @@ void DatabaseMaterializedMySQL::alterTable(ContextPtr context_, const StorageID void DatabaseMaterializedMySQL::drop(ContextPtr context_) { + LOG_TRACE(log, "Dropping MaterializeMySQL database"); /// Remove metadata info fs::path metadata(getMetadataPath() + "/.metadata"); diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 3698abf5542..60a88ea0d67 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 673bd155f77..c9e10b27caa 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -1,3 +1,4 @@ +#include "Common/logger_useful.h" #include "config.h" #if USE_MYSQL @@ -499,7 +500,10 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta { throw; } - catch (const mysqlxx::ConnectionFailed &) {} + catch (const mysqlxx::ConnectionFailed & ex) + { + LOG_TRACE(log, "Connection to MySQL failed {}", ex.displayText()); + } catch (const mysqlxx::BadQuery & e) { // Lost connection to MySQL server during query diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 1fd09f733f0..5096e0a03b7 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -1,17 +1,11 @@ -import os -import os.path as p import time -import pwd -import re import pymysql.cursors import pytest from helpers.cluster import ( ClickHouseCluster, ClickHouseInstance, get_docker_compose_path, - run_and_check, ) -import docker import logging from . import materialized_with_ddl @@ -63,8 +57,6 @@ class MySQLConnection: user="root", password="clickhouse", ip_address=None, - docker_compose=None, - project_name=cluster.project_name, ): self.user = user self.port = port From 8f5d0c0cd4b150d32a1257d18ee77cd582ac43cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 13:03:04 +0000 Subject: [PATCH 868/871] Revert "Merge pull request #52514 from azat/tests/fix-test_version_update_after_mutation" This reverts commit dd491eeae2e1ee030f4db5d3f0837067a2ee0ca8, reversing changes made to b225f9c34bf6064d246ede33f98e3c573fcda142. --- tests/integration/helpers/cluster.py | 9 --------- .../force_remove_data_recursively_on_drop.xml | 7 ------- .../test_version_update_after_mutation/test.py | 13 +++---------- 3 files changed, 3 insertions(+), 26 deletions(-) delete mode 100644 tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0448eb2437f..eff44de842a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3199,7 +3199,6 @@ class ClickHouseInstance: ): self.name = name self.base_cmd = cluster.base_cmd - self.base_dir = base_path self.docker_id = cluster.get_instance_docker_id(self.name) self.cluster = cluster self.hostname = hostname if hostname is not None else self.name @@ -4194,14 +4193,6 @@ class ClickHouseInstance: ["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"] ) - def put_users_config(self, config_path): - """Put new config (useful if you cannot put it at the start)""" - - instance_config_dir = p.abspath(p.join(self.path, "configs")) - users_d_dir = p.abspath(p.join(instance_config_dir, "users.d")) - config_path = p.join(self.base_dir, config_path) - shutil.copy(config_path, users_d_dir) - def create_dir(self): """Create the instance directory and all the needed files there.""" diff --git a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml deleted file mode 100644 index 7a00648b28e..00000000000 --- a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 416220c93c3..c80205d48c1 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -51,12 +51,6 @@ def start_cluster(): cluster.shutdown() -def restart_node(node): - # set force_remove_data_recursively_on_drop (cannot be done before, because the version is too old) - node.put_users_config("configs/force_remove_data_recursively_on_drop.xml") - node.restart_with_latest_version(signal=9, fix_metadata=True) - - def test_mutate_and_upgrade(start_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS mt") @@ -73,9 +67,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader - - restart_node(node1) - restart_node(node2) + node1.restart_with_latest_version(signal=9, fix_metadata=True) + node2.restart_with_latest_version(signal=9, fix_metadata=True) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -131,7 +124,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - restart_node(node3) + node3.restart_with_latest_version(signal=9, fix_metadata=True) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From b98d54cdf3a2f931040067418797336c6fb218a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 13:05:53 +0000 Subject: [PATCH 869/871] Add flushing logs --- tests/integration/test_version_update_after_mutation/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index c80205d48c1..eac214ea99f 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -67,6 +67,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader + node1.query("SYSTEM FLUSH LOGS") + node2.query("SYSTEM FLUSH LOGS") node1.restart_with_latest_version(signal=9, fix_metadata=True) node2.restart_with_latest_version(signal=9, fix_metadata=True) From c4a00b8c68ce39c4ee325ab8b29ea86e58dea8af Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 15:39:07 +0000 Subject: [PATCH 870/871] Throw S3Exception whenever possible. --- src/Backups/BackupIO_S3.cpp | 6 +++--- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/S3/copyS3File.cpp | 2 +- src/IO/S3/getObjectInfo.cpp | 2 +- src/Storages/DataLakes/S3MetadataReader.cpp | 4 ++-- src/Storages/StorageS3.cpp | 4 ++-- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index d487ec6e80e..40a043be552 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -88,7 +88,7 @@ namespace request.SetMaxKeys(1); auto outcome = client.ListObjects(request); if (!outcome.IsSuccess()) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); return outcome.GetResult().GetContents(); } @@ -271,7 +271,7 @@ void BackupWriterS3::removeFile(const String & file_name) request.SetKey(fs::path(s3_uri.key) / file_name); auto outcome = client->DeleteObject(request); if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType())) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } void BackupWriterS3::removeFiles(const Strings & file_names) @@ -329,7 +329,7 @@ void BackupWriterS3::removeFilesBatch(const Strings & file_names) auto outcome = client->DeleteObjects(request); if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType())) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e46ca3d0828..4da7b3e892f 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -135,7 +135,7 @@ private: return result; } - throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", quoteString(request.GetBucket()), quoteString(request.GetPrefix()), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 2de2ccd0f9f..dda107840cb 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -783,7 +783,7 @@ namespace if (!outcome.IsSuccess()) { abortMultipartUpload(); - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } return outcome.GetResult().GetCopyPartResult().GetETag(); diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index c652f16ab20..88f79f8d8d5 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -85,7 +85,7 @@ ObjectInfo getObjectInfo( } else if (throw_on_error) { - throw DB::Exception(ErrorCodes::S3_ERROR, + throw S3Exception(error.GetErrorType(), "Failed to get object info: {}. HTTP response code: {}", error.GetMessage(), static_cast(error.GetResponseCode())); } diff --git a/src/Storages/DataLakes/S3MetadataReader.cpp b/src/Storages/DataLakes/S3MetadataReader.cpp index f62c440bc2f..ac472c190e4 100644 --- a/src/Storages/DataLakes/S3MetadataReader.cpp +++ b/src/Storages/DataLakes/S3MetadataReader.cpp @@ -57,8 +57,8 @@ std::vector S3DataLakeMetadataReadHelper::listFiles( { outcome = client->ListObjectsV2(request); if (!outcome.IsSuccess()) - throw Exception( - ErrorCodes::S3_ERROR, + throw S3Exception( + outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", quoteString(bucket), quoteString(base_configuration.url.key), diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b52150250b8..ebce3a7aeca 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -245,7 +245,7 @@ private: if (!outcome.IsSuccess()) { - throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", quoteString(request.GetBucket()), quoteString(request.GetPrefix()), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } @@ -1195,7 +1195,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, if (!response.IsSuccess()) { const auto & err = response.GetError(); - throw Exception(ErrorCodes::S3_ERROR, "{}: {}", std::to_string(static_cast(err.GetErrorType())), err.GetMessage()); + throw S3Exception(err.GetMessage(), err.GetErrorType()); } for (const auto & error : response.GetResult().GetErrors()) From 4894a357f5a39943db1a95a65b3da507e82df83e Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 31 Jul 2023 17:49:59 -0600 Subject: [PATCH 871/871] Update datetime.md Correct URLs for settings-formats.md to date_time_input_format and date_time_output_format --- docs/en/sql-reference/data-types/datetime.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index fe279edb709..c99c8791542 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -140,8 +140,8 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) - [Functions for working with arrays](../../sql-reference/functions/array-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#settings-date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime)