From e77d1076b8bc0a5163a07966d26010fb4461ffe4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Feb 2024 23:03:22 +0100 Subject: [PATCH 01/85] Fix crash in libunwind while interpreting debug info --- contrib/libunwind | 2 +- src/Common/examples/CMakeLists.txt | 3 ++ src/Common/examples/check_pointer_valid.cpp | 52 +++++++++++++++++++++ src/Daemon/BaseDaemon.cpp | 28 +++++++++++ 4 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 src/Common/examples/check_pointer_valid.cpp diff --git a/contrib/libunwind b/contrib/libunwind index 40d8eadf96b..e4e1c06f4bc 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 40d8eadf96b127d9b22d53ce7a4fc52aaedea965 +Subproject commit e4e1c06f4bc31adb3fa20e517779171b4b521c78 diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 90a238c9800..73e1396fb35 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -87,3 +87,6 @@ if (ENABLE_SSL) clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) target_link_libraries (encrypt_decrypt PRIVATE dbms) endif() + +clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp) +target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io) diff --git a/src/Common/examples/check_pointer_valid.cpp b/src/Common/examples/check_pointer_valid.cpp new file mode 100644 index 00000000000..4baf8f2f01d --- /dev/null +++ b/src/Common/examples/check_pointer_valid.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include + + +/// This example demonstrates how is it possible to check if a pointer to memory is readable using a signal handler. + +thread_local bool checking_pointer = false; +thread_local jmp_buf signal_jump_buffer; + + +void signalHandler(int sig, siginfo_t *, void *) +{ + if (checking_pointer && sig == SIGSEGV) + siglongjmp(signal_jump_buffer, 1); +} + +bool isPointerValid(const void * ptr) +{ + checking_pointer = true; + if (0 == sigsetjmp(signal_jump_buffer, 1)) + { + char res; + memcpy(&res, ptr, 1); + __asm__ __volatile__("" :: "r"(res) : "memory"); + return true; + } + else + { + checking_pointer = false; + return false; + } +} + +int main(int, char **) +{ + struct sigaction sa; + memset(&sa, 0, sizeof(sa)); + sa.sa_sigaction = signalHandler; + sa.sa_flags = SA_SIGINFO; + + if (sigemptyset(&sa.sa_mask) + || sigaddset(&sa.sa_mask, SIGSEGV) + || sigaction(SIGSEGV, &sa, nullptr)) + return 1; + + std::cerr << isPointerValid(reinterpret_cast(0x123456789)) << "\n"; + std::cerr << isPointerValid(&sa) << "\n"; + + return 0; +} diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 289a41bb75e..0e781baf0d4 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -137,10 +137,18 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) static std::atomic_flag fatal_error_printed; +/// Special handling for a function isPointerReadable. +thread_local bool checking_pointer = false; +thread_local jmp_buf signal_jump_buffer; + /** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ static void signalHandler(int sig, siginfo_t * info, void * context) { + /// Special handling for a function isPointerReadable. + if (checking_pointer && sig == SIGSEGV) + siglongjmp(signal_jump_buffer, 1); + DENY_ALLOCATIONS_IN_SCOPE; auto saved_errno = errno; /// We must restore previous value of errno in signal handler. @@ -182,6 +190,26 @@ static void signalHandler(int sig, siginfo_t * info, void * context) errno = saved_errno; } +/// This function can be used from other translation units, +/// For example, from libunwind while parsing debug info, which is unsafe. +/// Note: we are checking only the first byte, which is ok for aligned words. +extern "C" bool isPointerReadable(const void * ptr) +{ + checking_pointer = true; + if (0 == sigsetjmp(signal_jump_buffer, 1)) + { + char res; + memcpy(&res, ptr, 1); + __asm__ __volatile__("" :: "r"(res) : "memory"); + return true; + } + else + { + checking_pointer = false; + return false; + } +} + static bool getenvBool(const char * name) { bool res = false; From 676519ce90a5cf3e39415885ad72c09d5202b09a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Feb 2024 23:06:47 +0100 Subject: [PATCH 02/85] Fix crash in libunwind while interpreting debug info --- src/Common/examples/check_pointer_valid.cpp | 1 + src/Daemon/BaseDaemon.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Common/examples/check_pointer_valid.cpp b/src/Common/examples/check_pointer_valid.cpp index 4baf8f2f01d..a6b6ee7027d 100644 --- a/src/Common/examples/check_pointer_valid.cpp +++ b/src/Common/examples/check_pointer_valid.cpp @@ -24,6 +24,7 @@ bool isPointerValid(const void * ptr) char res; memcpy(&res, ptr, 1); __asm__ __volatile__("" :: "r"(res) : "memory"); + checking_pointer = false; return true; } else diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 0e781baf0d4..2ca51ae72e0 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -201,6 +201,7 @@ extern "C" bool isPointerReadable(const void * ptr) char res; memcpy(&res, ptr, 1); __asm__ __volatile__("" :: "r"(res) : "memory"); + checking_pointer = false; return true; } else From b1e67c7bc6a45dd06e27b44a56531bca1745268e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 28 Feb 2024 09:55:16 +0000 Subject: [PATCH 03/85] s/jmp_buf/sigjmp_buf/ --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 2ca51ae72e0..b6ebe80b07d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -139,7 +139,7 @@ static std::atomic_flag fatal_error_printed; /// Special handling for a function isPointerReadable. thread_local bool checking_pointer = false; -thread_local jmp_buf signal_jump_buffer; +thread_local sigjmp_buf signal_jump_buffer; /** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ From a95e7de257455a95855859c7f3b8d64583025027 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 02:44:55 +0100 Subject: [PATCH 04/85] Support for MSan --- src/Daemon/BaseDaemon.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index b6ebe80b07d..b7e0f765799 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -200,6 +200,7 @@ extern "C" bool isPointerReadable(const void * ptr) { char res; memcpy(&res, ptr, 1); + __msan_unpoison(res, 1); __asm__ __volatile__("" :: "r"(res) : "memory"); checking_pointer = false; return true; From a67d862bd02dfea8be4fad1bcc984fe6194c021d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 02:47:15 +0100 Subject: [PATCH 05/85] Support for sanitizers --- src/Daemon/BaseDaemon.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index b7e0f765799..c167ca1baa0 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -190,6 +190,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) errno = saved_errno; } +#if !defined(SANITIZER) /// This function can be used from other translation units, /// For example, from libunwind while parsing debug info, which is unsafe. /// Note: we are checking only the first byte, which is ok for aligned words. @@ -200,7 +201,6 @@ extern "C" bool isPointerReadable(const void * ptr) { char res; memcpy(&res, ptr, 1); - __msan_unpoison(res, 1); __asm__ __volatile__("" :: "r"(res) : "memory"); checking_pointer = false; return true; @@ -211,6 +211,7 @@ extern "C" bool isPointerReadable(const void * ptr) return false; } } +#endif static bool getenvBool(const char * name) { From 1899a3062e9f60e8dc07984fb10113c7dcc2e72c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 02:48:31 +0100 Subject: [PATCH 06/85] Clang-tidy --- src/Common/examples/check_pointer_valid.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/examples/check_pointer_valid.cpp b/src/Common/examples/check_pointer_valid.cpp index a6b6ee7027d..e59ebf43327 100644 --- a/src/Common/examples/check_pointer_valid.cpp +++ b/src/Common/examples/check_pointer_valid.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include From 784b656c6a8d19cacc184ff923bbff4d27882c5b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Mar 2024 00:44:36 +0100 Subject: [PATCH 07/85] Another approach --- contrib/libunwind | 2 +- src/Common/ProfileEvents.cpp | 1 + src/Common/QueryProfiler.cpp | 26 ++++++++++++++++++++++---- src/Common/StackTrace.cpp | 4 ++++ src/Common/StackTrace.h | 6 ++++++ src/Daemon/BaseDaemon.cpp | 31 ++----------------------------- 6 files changed, 36 insertions(+), 34 deletions(-) diff --git a/contrib/libunwind b/contrib/libunwind index e4e1c06f4bc..5c221df716e 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit e4e1c06f4bc31adb3fa20e517779171b4b521c78 +Subproject commit 5c221df716eaad6b5637bfdae593296db18d23a0 diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c1ac3d08245..4b66a920abb 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -361,6 +361,7 @@ The server successfully detected this situation and will download merged part fr M(QueryProfilerSignalOverruns, "Number of times we drop processing of a query profiler signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \ M(QueryProfilerConcurrencyOverruns, "Number of times we drop processing of a query profiler signal due to too many concurrent query profilers in other threads, which may indicate overload.") \ M(QueryProfilerRuns, "Number of times QueryProfiler had been run.") \ + M(QueryProfilerErrors, "Invalid memory accesses during asynchronous stack unwinding.") \ \ M(CreatedLogEntryForMerge, "Successfully created log entry to merge parts in ReplicatedMergeTree.") \ M(NotCreatedLogEntryForMerge, "Log entry to merge parts in ReplicatedMergeTree is not created due to concurrent log update by another replica.") \ diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 34ffbf6c498..5ec784527f7 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -11,7 +11,6 @@ #include #include -#include namespace CurrentMetrics { @@ -24,6 +23,7 @@ namespace ProfileEvents extern const Event QueryProfilerSignalOverruns; extern const Event QueryProfilerConcurrencyOverruns; extern const Event QueryProfilerRuns; + extern const Event QueryProfilerErrors; } namespace DB @@ -83,11 +83,29 @@ namespace #endif const auto signal_context = *reinterpret_cast(context); - const StackTrace stack_trace(signal_context); + std::optional stack_trace; + +#if defined(SANITIZER) + constexpr bool sanitizer = true; +#else + constexpr bool sanitizer = false; +#endif + + asynchronous_stack_unwinding = true; + if (sanitizer || 0 == sigsetjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1)) + { + stack_trace.emplace(signal_context); + } + else + { + ProfileEvents::incrementNoTrace(ProfileEvents::QueryProfilerErrors); + } + asynchronous_stack_unwinding = false; + + if (stack_trace) + TraceSender::send(trace_type, *stack_trace, {}); - TraceSender::send(trace_type, stack_trace, {}); ProfileEvents::incrementNoTrace(ProfileEvents::QueryProfilerRuns); - errno = saved_errno; } diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 436b85ff30b..538aad5a79a 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -532,3 +532,7 @@ void StackTrace::dropCache() std::lock_guard lock{stacktrace_cache_mutex}; cacheInstance().clear(); } + + +thread_local bool asynchronous_stack_unwinding = false; +thread_local sigjmp_buf asynchronous_stack_unwinding_signal_jump_buffer; diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index a16d889a67a..1d72b0239a6 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -8,6 +8,7 @@ #include #include #include +#include #ifdef OS_DARWIN // ucontext is not available without _XOPEN_SOURCE @@ -87,3 +88,8 @@ protected: }; std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context); + +/// Special handling for errors during asynchronous stack unwinding, +/// Which is used in Query Profiler +extern thread_local bool asynchronous_stack_unwinding; +extern thread_local sigjmp_buf asynchronous_stack_unwinding_signal_jump_buffer; diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index c167ca1baa0..3b030c918e7 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -137,17 +137,12 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) static std::atomic_flag fatal_error_printed; -/// Special handling for a function isPointerReadable. -thread_local bool checking_pointer = false; -thread_local sigjmp_buf signal_jump_buffer; - /** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ static void signalHandler(int sig, siginfo_t * info, void * context) { - /// Special handling for a function isPointerReadable. - if (checking_pointer && sig == SIGSEGV) - siglongjmp(signal_jump_buffer, 1); + if (asynchronous_stack_unwinding && sig == SIGSEGV) + siglongjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1); DENY_ALLOCATIONS_IN_SCOPE; auto saved_errno = errno; /// We must restore previous value of errno in signal handler. @@ -190,28 +185,6 @@ static void signalHandler(int sig, siginfo_t * info, void * context) errno = saved_errno; } -#if !defined(SANITIZER) -/// This function can be used from other translation units, -/// For example, from libunwind while parsing debug info, which is unsafe. -/// Note: we are checking only the first byte, which is ok for aligned words. -extern "C" bool isPointerReadable(const void * ptr) -{ - checking_pointer = true; - if (0 == sigsetjmp(signal_jump_buffer, 1)) - { - char res; - memcpy(&res, ptr, 1); - __asm__ __volatile__("" :: "r"(res) : "memory"); - checking_pointer = false; - return true; - } - else - { - checking_pointer = false; - return false; - } -} -#endif static bool getenvBool(const char * name) { From 92ebf3d5becdf7af2bc17e3afb479fd456f008c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Mar 2024 09:22:14 +0100 Subject: [PATCH 08/85] Loosen --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index b55e9810361..9ffb4789dc9 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,8 +61,8 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # set CPU time limit to 1000 seconds set (RLIMIT_CPU 1000) - # -fsanitize=memory is too heavy - if (SANITIZE STREQUAL "memory") + # -fsanitize=memory and address are too heavy + if (SANITIZE) set (RLIMIT_DATA 10000000000) # 10G endif() From 3e68103ac84e0c3d59759f5af5d615191a6a59e5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Apr 2024 09:11:20 +0200 Subject: [PATCH 09/85] Fix interserver secret for Distributed over Distributed from remote() Right if you are executing remote() and later the query will go to the cluster with interserver secret, then you should have the same user on the nodes from that cluster, otherwise the query will fail with: DB::NetException: Connection reset by peer And on the remote node: TCPHandler: User (initial, interserver mode): new_user (client: 172.16.1.5:40536) TCP_INTERSERVER-Session: d29ecf7d-2c1c-44d2-8cc9-4ab08175bf05 Authentication failed with error: new_user: Authentication failed: password is incorrect, or there is no user with such name. ServerErrorHandler: Code: 516. DB::Exception: new_user: Authentication failed: password is incorrect, or there is no user with such name. (AUTHENTICATION_FAILED), Stack trace (when copying this message, always include the lines below): The problem is that remote() will not use passed to it user in any form, and instead, the initial user will be used, i.e. "cli_user" not "query_user": chc --user cli_user -q "select * from remote(node, default, some_dist_table, 'query_user')" Fix this by using the user from query for the remote(). Note, that the Distributed over Distributed in case of tables still wont work, for this you have to have the same users on all nodes in all clusters that are involved in case of interserver secret is enabled (see also test). Signed-off-by: Azat Khuzhin v2: move client initial_user adjustment into ClusterProxy/executeQuery.cpp v3: we cannot check for interserver_mode in updateSettingsAndClientInfoForCluster() since it is not yet interserver in remote() context --- .../ClusterProxy/executeQuery.cpp | 32 ++++++++++++-- src/Interpreters/ClusterProxy/executeQuery.h | 11 ++--- src/Storages/StorageDistributed.cpp | 3 +- .../configs/users.d/new_user.xml | 12 +++++ .../test.py | 44 ++++++++++++++++--- 5 files changed, 83 insertions(+), 19 deletions(-) create mode 100644 tests/integration/test_distributed_inter_server_secret/configs/users.d/new_user.xml diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index f7727f70ff7..2af33421add 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -38,7 +38,8 @@ namespace ErrorCodes namespace ClusterProxy { -ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, +ContextMutablePtr updateSettingsAndClientInfoForCluster(const Cluster & cluster, + bool is_remote_function, ContextPtr context, const Settings & settings, const StorageID & main_table, @@ -46,9 +47,17 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, LoggerPtr log, const DistributedSettings * distributed_settings) { + ClientInfo new_client_info = context->getClientInfo(); Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); + /// In case of interserver mode we should reset initial_user for remote() function to use passed user from the query. + if (is_remote_function) + { + const auto & address = cluster.getShardsAddresses().front().front(); + new_client_info.initial_user = address.user; + } + /// 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. @@ -168,9 +177,23 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); + new_context->setClientInfo(new_client_info); return new_context; } +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table) +{ + return updateSettingsAndClientInfoForCluster(cluster, + /* is_remote_function= */ false, + context, + settings, + main_table, + /* additional_filter_ast= */ {}, + /* log= */ {}, + /* distributed_settings= */ {}); +} + + static ThrottlerPtr getThrottler(const ContextPtr & context) { const Settings & settings = context->getSettingsRef(); @@ -209,7 +232,8 @@ void executeQuery( const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, - AdditionalShardFilterGenerator shard_filter_generator) + AdditionalShardFilterGenerator shard_filter_generator, + bool is_remote_function) { const Settings & settings = context->getSettingsRef(); @@ -222,8 +246,8 @@ void executeQuery( SelectStreamFactory::Shards remote_shards; auto cluster = query_info.getCluster(); - auto new_context = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log, - &distributed_settings); + auto new_context = updateSettingsAndClientInfoForCluster(*cluster, is_remote_function, context, + settings, main_table, query_info.additional_filter_ast, log, &distributed_settings); if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas && context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value != new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 8f6f6300c7b..3734a237d19 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -38,13 +38,7 @@ 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, - ASTPtr additional_filter_ast = nullptr, - LoggerPtr log = nullptr, - const DistributedSettings * distributed_settings = nullptr); +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table); using AdditionalShardFilterGenerator = std::function; /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. @@ -63,7 +57,8 @@ void executeQuery( const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, - AdditionalShardFilterGenerator shard_filter_generator); + AdditionalShardFilterGenerator shard_filter_generator, + bool is_remote_function); void executeQueryWithParallelReplicas( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 69d3cf3ad3b..a4f51f1f587 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -927,7 +927,8 @@ void StorageDistributed::read( sharding_key_expr, sharding_key_column_name, distributed_settings, - additional_shard_filter_generator); + additional_shard_filter_generator, + /* is_remote_function= */ static_cast(owned_cluster)); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) diff --git a/tests/integration/test_distributed_inter_server_secret/configs/users.d/new_user.xml b/tests/integration/test_distributed_inter_server_secret/configs/users.d/new_user.xml new file mode 100644 index 00000000000..a747d61a0dd --- /dev/null +++ b/tests/integration/test_distributed_inter_server_secret/configs/users.d/new_user.xml @@ -0,0 +1,12 @@ + + + + + + ::/0 + + default + default + + + diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 10dbb23d961..50d7be4d11e 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -12,12 +12,16 @@ from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) -def make_instance(name, cfg, *args, **kwargs): +def make_instance(name, *args, **kwargs): + main_configs = kwargs.pop("main_configs", []) + main_configs.append("configs/remote_servers.xml") + user_configs = kwargs.pop("user_configs", []) + user_configs.append("configs/users.xml") return cluster.add_instance( name, with_zookeeper=True, - main_configs=["configs/remote_servers.xml", cfg], - user_configs=["configs/users.xml"], + main_configs=main_configs, + user_configs=user_configs, *args, **kwargs, ) @@ -27,11 +31,16 @@ def make_instance(name, cfg, *args, **kwargs): assert CLICKHOUSE_CI_MIN_TESTED_VERSION < "23.3" # _n1/_n2 contains cluster with different -- should fail -n1 = make_instance("n1", "configs/remote_servers_n1.xml") -n2 = make_instance("n2", "configs/remote_servers_n2.xml") +# only n1 contains new_user +n1 = make_instance( + "n1", + main_configs=["configs/remote_servers_n1.xml"], + user_configs=["configs/users.d/new_user.xml"], +) +n2 = make_instance("n2", main_configs=["configs/remote_servers_n2.xml"]) backward = make_instance( "backward", - "configs/remote_servers_backward.xml", + main_configs=["configs/remote_servers_backward.xml"], image="clickhouse/clickhouse-server", # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, @@ -100,6 +109,12 @@ def bootstrap(): ) """ ) + n.query( + """ + CREATE TABLE dist_over_dist_secure AS data + Engine=Distributed(secure, currentDatabase(), dist_secure, key) + """ + ) @pytest.fixture(scope="module", autouse=True) @@ -432,3 +447,20 @@ def test_user_secure_cluster_from_backward(user, password): assert n1.contains_in_log( "Using deprecated interserver protocol because the client is too old. Consider upgrading all nodes in cluster." ) + + +def test_secure_cluster_distributed_over_distributed_different_users(): + # This works because we will have initial_user='default' + n1.query( + "SELECT * FROM remote('n1', currentDatabase(), dist_secure)", user="new_user" + ) + # While this is broken because now initial_user='new_user', and n2 does not has it + with pytest.raises(QueryRuntimeException): + n2.query( + "SELECT * FROM remote('n1', currentDatabase(), dist_secure, 'new_user')" + ) + # And this is still a problem, let's assume that this is OK, since we are + # expecting that in case of dist-over-dist the clusters are the same (users + # and stuff). + with pytest.raises(QueryRuntimeException): + n1.query("SELECT * FROM dist_over_dist_secure", user="new_user") From ed63ad5e61916f38fa8db2143aaa755d49665584 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Tue, 7 May 2024 14:10:49 +0300 Subject: [PATCH 10/85] iouring: refactor get from context --- src/Coordination/Standalone/Context.cpp | 3 +- src/Disks/IO/IOUringReader.cpp | 2 +- src/Disks/IO/createReadBufferFromFileBase.cpp | 11 +---- src/Disks/IO/getIOUringReader.cpp | 40 +++++++++++++++++++ src/Disks/IO/getIOUringReader.h | 19 +++++++++ src/Interpreters/Context.cpp | 3 +- src/Storages/StorageFile.cpp | 6 +-- 7 files changed, 68 insertions(+), 16 deletions(-) create mode 100644 src/Disks/IO/getIOUringReader.cpp create mode 100644 src/Disks/IO/getIOUringReader.h diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 1095a11566f..84e54ed7100 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -306,7 +307,7 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co IOUringReader & Context::getIOURingReader() const { callOnce(shared->io_uring_reader_initialized, [&] { - shared->io_uring_reader = std::make_unique(512); + shared->io_uring_reader = createIOUringReader(); }); return *shared->io_uring_reader; diff --git a/src/Disks/IO/IOUringReader.cpp b/src/Disks/IO/IOUringReader.cpp index 90a4d285ecb..6b0e3f8cc89 100644 --- a/src/Disks/IO/IOUringReader.cpp +++ b/src/Disks/IO/IOUringReader.cpp @@ -1,5 +1,4 @@ #include "IOUringReader.h" -#include #if USE_LIBURING @@ -13,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents { diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index a9d451496ff..f3bb6ae1740 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -4,9 +4,9 @@ #include #include #include +#include #include #include -#include #include #include #include "config.h" @@ -100,14 +100,7 @@ std::unique_ptr createReadBufferFromFileBase( else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { #if USE_LIBURING - auto global_context = Context::getGlobalContextInstance(); - if (!global_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot obtain io_uring reader (global context not initialized)"); - - auto & reader = global_context->getIOURingReader(); - if (!reader.isSupported()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); - + auto & reader = getIOURingReaderOrThrow(); res = std::make_unique( reader, settings.priority, diff --git a/src/Disks/IO/getIOUringReader.cpp b/src/Disks/IO/getIOUringReader.cpp new file mode 100644 index 00000000000..8e9a9655a41 --- /dev/null +++ b/src/Disks/IO/getIOUringReader.cpp @@ -0,0 +1,40 @@ +#include "getIOUringReader.h" + +#if USE_LIBURING + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_METHOD; +} + +std::unique_ptr createIOUringReader() +{ + return std::make_unique(512); +} + +IOUringReader & getIOUringReaderOrThrow(ContextPtr context) +{ + auto reader = context->getIOUringReader(); + if (!reader.isSupported) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); + } + return reader; +} + +IOUringReader & getIOUringReaderOrThrow() +{ + auto context = Context::getGlobalContextInstance(); + if (!context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized"); + return getIOUringReaderOrThrow(context) +} + +} +#endif diff --git a/src/Disks/IO/getIOUringReader.h b/src/Disks/IO/getIOUringReader.h new file mode 100644 index 00000000000..0980f32b5a2 --- /dev/null +++ b/src/Disks/IO/getIOUringReader.h @@ -0,0 +1,19 @@ +#pragma once + +#include "config.h" + +#if USE_LIBURING + +#include + +namespace DB +{ + +std::unique_ptr createIOUringReader(); + +IOUringReader & getIOUringReaderOrThrow(ContextPtr); + +IOUringReader & getIOUringReaderOrThrow(); + +} +#endif diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 44d36e94441..d847cab013c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -5178,7 +5179,7 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co IOUringReader & Context::getIOURingReader() const { callOnce(shared->io_uring_reader_initialized, [&] { - shared->io_uring_reader = std::make_unique(512); + shared->io_uring_reader = createIOUringReader() }); return *shared->io_uring_reader; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 430e68d8562..9bead6d0267 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -282,10 +283,7 @@ std::unique_ptr selectReadBuffer( else if (read_method == LocalFSReadMethod::io_uring && !use_table_fd) { #if USE_LIBURING - auto & reader = context->getIOURingReader(); - if (!reader.isSupported()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); - + auto & reader = getIOURingReaderOrThrow(context); res = std::make_unique( reader, Priority{}, From f57abbd806ef78be7829844e3f285b994661ca5e Mon Sep 17 00:00:00 2001 From: tomershafir Date: Tue, 7 May 2024 17:27:52 +0300 Subject: [PATCH 11/85] add missing include --- src/Disks/IO/getIOUringReader.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/IO/getIOUringReader.h b/src/Disks/IO/getIOUringReader.h index 0980f32b5a2..ca619785ab4 100644 --- a/src/Disks/IO/getIOUringReader.h +++ b/src/Disks/IO/getIOUringReader.h @@ -5,6 +5,7 @@ #if USE_LIBURING #include +#include namespace DB { From eedef6d826412b6b9f621527a0c4367142b03632 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 7 May 2024 11:10:27 +0300 Subject: [PATCH 12/85] fix empty used_dictionaries in query_log --- .../ExternalDictionariesLoader.cpp | 4 +- ...3148_query_log_used_dictionaries.reference | 4 + .../03148_query_log_used_dictionaries.sql | 84 +++++++++++++++++++ 3 files changed, 90 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03148_query_log_used_dictionaries.reference create mode 100644 tests/queries/0_stateless/03148_query_log_used_dictionaries.sql diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index f48ee61dab8..49891e6cd60 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -79,7 +79,7 @@ ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(co std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) - local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); return std::static_pointer_cast(load(resolved_dictionary_name)); } @@ -89,7 +89,7 @@ ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) - local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); } diff --git a/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference b/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference new file mode 100644 index 00000000000..1f54474efa3 --- /dev/null +++ b/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference @@ -0,0 +1,4 @@ +simple_with_analyzer 1 +nested_with_analyzer 1 +simple_without_analyzer 1 +nested_without_analyzer 1 diff --git a/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql new file mode 100644 index 00000000000..6f10b118c92 --- /dev/null +++ b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql @@ -0,0 +1,84 @@ +DROP DICTIONARY IF EXISTS 03148_dictionary; + +CREATE DICTIONARY 03148_dictionary ( + id UInt64, + name String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE( + QUERY 'select 0 as id, ''name0'' as name' +)) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED); + +SELECT + dictGet('03148_dictionary', 'name', number) as dict_value +FROM numbers(1) +SETTINGS + allow_experimental_analyzer = 1, + log_comment = 'simple_with_analyzer' +FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT 'simple_with_analyzer', length(used_dictionaries) as used_dictionaries_qty +FROM system.query_log +WHERE current_database = currentDatabase() + AND type = 'QueryFinish' + AND log_comment = 'simple_with_analyzer'; + +SELECT * +FROM ( + SELECT + dictGet('03148_dictionary', 'name', number) as dict_value + FROM numbers(1) +) t +SETTINGS + allow_experimental_analyzer = 1, + log_comment = 'nested_with_analyzer' +FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT 'nested_with_analyzer', length(used_dictionaries) as used_dictionaries_qty +FROM system.query_log +WHERE current_database = currentDatabase() + AND type = 'QueryFinish' + AND log_comment = 'nested_with_analyzer'; + +SELECT + dictGet('03148_dictionary', 'name', number) as dict_value +FROM numbers(1) +SETTINGS + allow_experimental_analyzer = 0, + log_comment = 'simple_without_analyzer' +FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT 'simple_without_analyzer', length(used_dictionaries) as used_dictionaries_qty +FROM system.query_log +WHERE current_database = currentDatabase() + AND type = 'QueryFinish' + AND log_comment = 'simple_without_analyzer'; + +SELECT * +FROM ( + SELECT + dictGet('03148_dictionary', 'name', number) as dict_value + FROM numbers(1) +) t +SETTINGS + allow_experimental_analyzer = 0, + log_comment = 'nested_without_analyzer' +FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT 'nested_without_analyzer', length(used_dictionaries) as used_dictionaries_qty +FROM system.query_log +WHERE current_database = currentDatabase() + AND type = 'QueryFinish' + AND log_comment = 'nested_without_analyzer'; + +DROP DICTIONARY IF EXISTS 03148_dictionary; From a365c36e9d4202d1f1a7802268b27ac5b35673b9 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Thu, 9 May 2024 11:06:31 +0300 Subject: [PATCH 13/85] use qualified dictionary name in query log --- docs/en/operations/system-tables/query_log.md | 2 +- src/Dictionaries/IDictionary.h | 9 +++++++++ src/Interpreters/ExternalDictionariesLoader.cpp | 12 +++++++----- .../03148_query_log_used_dictionaries.reference | 8 ++++---- .../03148_query_log_used_dictionaries.sql | 8 ++++---- 5 files changed, 25 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index d48eb31df00..75b855966a3 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -108,7 +108,7 @@ Columns: - `used_aggregate_function_combinators` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `aggregate functions combinators`, which were used during query execution. - `used_database_engines` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `database engines`, which were used during query execution. - `used_data_type_families` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `data type families`, which were used during query execution. -- `used_dictionaries` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `dictionaries`, which were used during query execution. +- `used_dictionaries` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `dictionaries`, which were used during query execution. For dictionaries configured using an XML file this is the name of the dictionary, and for dictionaries created by an SQL statement, the canonical name is the fully qualified object name. - `used_formats` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `formats`, which were used during query execution. - `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. diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index bab80d3cd57..944e00f14c9 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -69,6 +69,15 @@ public: return dictionary_id.getNameForLogs(); } + /// Returns fully qualified unquoted dictionary name + std::string getQualifiedName() const + { + std::lock_guard lock{mutex}; + if (dictionary_id.database_name.empty()) + return dictionary_id.table_name; + return dictionary_id.database_name + "." + dictionary_id.table_name; + } + StorageID getDictionaryID() const { std::lock_guard lock{mutex}; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 49891e6cd60..1685c06d387 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -77,21 +77,23 @@ void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExterna ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); + auto dictionary = std::static_pointer_cast(load(resolved_dictionary_name)); if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) - local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, dictionary->getQualifiedName()); - return std::static_pointer_cast(load(resolved_dictionary_name)); + return dictionary; } ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); + auto dictionary = std::static_pointer_cast(tryLoad(resolved_dictionary_name)); - if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) - local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries && dictionary) + local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, dictionary->getQualifiedName()); - return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); + return dictionary; } diff --git a/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference b/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference index 1f54474efa3..4fa3a14e63f 100644 --- a/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference +++ b/tests/queries/0_stateless/03148_query_log_used_dictionaries.reference @@ -1,4 +1,4 @@ -simple_with_analyzer 1 -nested_with_analyzer 1 -simple_without_analyzer 1 -nested_without_analyzer 1 +simple_with_analyzer ['default.03148_dictionary'] +nested_with_analyzer ['default.03148_dictionary'] +simple_without_analyzer ['default.03148_dictionary'] +nested_without_analyzer ['default.03148_dictionary'] diff --git a/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql index 6f10b118c92..1b647a7ee62 100644 --- a/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql +++ b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql @@ -21,7 +21,7 @@ FORMAT Null; SYSTEM FLUSH LOGS; -SELECT 'simple_with_analyzer', length(used_dictionaries) as used_dictionaries_qty +SELECT log_comment, used_dictionaries FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' @@ -40,7 +40,7 @@ FORMAT Null; SYSTEM FLUSH LOGS; -SELECT 'nested_with_analyzer', length(used_dictionaries) as used_dictionaries_qty +SELECT log_comment, used_dictionaries FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' @@ -56,7 +56,7 @@ FORMAT Null; SYSTEM FLUSH LOGS; -SELECT 'simple_without_analyzer', length(used_dictionaries) as used_dictionaries_qty +SELECT log_comment, used_dictionaries FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' @@ -75,7 +75,7 @@ FORMAT Null; SYSTEM FLUSH LOGS; -SELECT 'nested_without_analyzer', length(used_dictionaries) as used_dictionaries_qty +SELECT log_comment, used_dictionaries FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' From b97bf456c6e89f36d5225c9befda3738f54cdf31 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Wed, 8 May 2024 18:51:25 +0300 Subject: [PATCH 14/85] try fix build --- programs/keeper/CMakeLists.txt | 1 + src/Coordination/Standalone/Context.cpp | 2 +- src/Coordination/Standalone/Context.h | 2 +- src/Disks/IO/createReadBufferFromFileBase.cpp | 4 ++-- src/Disks/IO/getIOUringReader.cpp | 9 +++++---- src/Disks/IO/getIOUringReader.h | 1 + src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 2 +- src/Storages/StorageFile.cpp | 2 +- 9 files changed, 15 insertions(+), 12 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index b811868333b..af360e44ff4 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -148,6 +148,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/createReadBufferFromFileBase.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getIOUringReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousBoundedReadBuffer.cpp diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 84e54ed7100..ec5400b7384 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -304,7 +304,7 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co } #if USE_LIBURING -IOUringReader & Context::getIOURingReader() const +IOUringReader & Context::getIOUringReader() const { callOnce(shared->io_uring_reader_initialized, [&] { shared->io_uring_reader = createIOUringReader(); diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index adb9111185f..29a66a0e3c7 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -137,7 +137,7 @@ public: IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; #if USE_LIBURING - IOUringReader & getIOURingReader() const; + IOUringReader & getIOUringReader() const; #endif std::shared_ptr getAsyncReadCounters() const; ThreadPool & getThreadPoolWriter() const; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index f3bb6ae1740..b132e25ac6b 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -100,7 +100,7 @@ std::unique_ptr createReadBufferFromFileBase( else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { #if USE_LIBURING - auto & reader = getIOURingReaderOrThrow(); + auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( reader, settings.priority, diff --git a/src/Disks/IO/getIOUringReader.cpp b/src/Disks/IO/getIOUringReader.cpp index 8e9a9655a41..d9cc6211164 100644 --- a/src/Disks/IO/getIOUringReader.cpp +++ b/src/Disks/IO/getIOUringReader.cpp @@ -1,7 +1,8 @@ -#include "getIOUringReader.h" +#include #if USE_LIBURING +#include #include namespace DB @@ -20,8 +21,8 @@ std::unique_ptr createIOUringReader() IOUringReader & getIOUringReaderOrThrow(ContextPtr context) { - auto reader = context->getIOUringReader(); - if (!reader.isSupported) + auto & reader = context->getIOUringReader(); + if (!reader.isSupported()) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); } @@ -33,7 +34,7 @@ IOUringReader & getIOUringReaderOrThrow() auto context = Context::getGlobalContextInstance(); if (!context) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized"); - return getIOUringReaderOrThrow(context) + return getIOUringReaderOrThrow(context); } } diff --git a/src/Disks/IO/getIOUringReader.h b/src/Disks/IO/getIOUringReader.h index ca619785ab4..59e71980750 100644 --- a/src/Disks/IO/getIOUringReader.h +++ b/src/Disks/IO/getIOUringReader.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d847cab013c..db374bc85f9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5176,10 +5176,10 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co } #if USE_LIBURING -IOUringReader & Context::getIOURingReader() const +IOUringReader & Context::getIOUringReader() const { callOnce(shared->io_uring_reader_initialized, [&] { - shared->io_uring_reader = createIOUringReader() + shared->io_uring_reader = createIOUringReader(); }); return *shared->io_uring_reader; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d1ff5b4c2b2..0430db10de2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1243,7 +1243,7 @@ public: IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; #if USE_LIBURING - IOUringReader & getIOURingReader() const; + IOUringReader & getIOUringReader() const; #endif std::shared_ptr getAsyncReadCounters() const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9bead6d0267..1493e649b60 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -283,7 +283,7 @@ std::unique_ptr selectReadBuffer( else if (read_method == LocalFSReadMethod::io_uring && !use_table_fd) { #if USE_LIBURING - auto & reader = getIOURingReaderOrThrow(context); + auto & reader = getIOUringReaderOrThrow(context); res = std::make_unique( reader, Priority{}, From 5f312eb3e3fea61bed2ed85a4d16f17ec9c565b6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 May 2024 09:53:03 +0200 Subject: [PATCH 15/85] Correctly handle max_block_size=0 in Numbers source --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 16 ++++++++++++---- .../03149_numbers_max_block_size_zero.reference | 2 ++ .../03149_numbers_max_block_size_zero.sql | 2 ++ 3 files changed, 16 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference create mode 100644 tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 11371578c79..759dc7354df 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -176,8 +176,9 @@ protected: { std::lock_guard lock(ranges_state->mutex); - UInt64 need = base_block_size_; + bool without_block_size_limit = need == 0; + UInt64 size = 0; /// how many item found. /// find start @@ -185,14 +186,21 @@ protected: end = start; /// find end - while (need != 0) + while (without_block_size_limit || need != 0) { UInt128 can_provide = end.offset_in_ranges == ranges.size() ? static_cast(0) : ranges[end.offset_in_ranges].size - end.offset_in_range; + if (can_provide == 0) break; - if (can_provide > need) + if (without_block_size_limit) + { + end.offset_in_ranges++; + end.offset_in_range = 0; + size += static_cast(can_provide); + } + else if (can_provide > need) { end.offset_in_range += need; size += need; @@ -527,7 +535,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() checkLimits(size_t(total_size)); - if (total_size / max_block_size < num_streams) + if (max_block_size != 0 && total_size / max_block_size < num_streams) num_streams = static_cast(total_size / max_block_size); if (num_streams == 0) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference new file mode 100644 index 00000000000..896f02d1185 --- /dev/null +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference @@ -0,0 +1,2 @@ +1320 +1320 diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql new file mode 100644 index 00000000000..afc4e4d57a5 --- /dev/null +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql @@ -0,0 +1,2 @@ +SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0; +SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 1; From 555e9ede91a7872e8f30173a05560bf9e83ec056 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 10 May 2024 20:19:32 +0200 Subject: [PATCH 16/85] Add anova documentation --- .../reference/analysis_of_variance.md | 44 +++++++++++++++++++ .../aggregate-functions/reference/index.md | 1 + 2 files changed, 45 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md b/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md new file mode 100644 index 00000000000..7f0df74010b --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/analysis_of_variance +sidebar_position: 6 +--- + +# analysisOfVariance + +Provides a statistical test for one-way analysis of variance (ANOVA test). It is a test over several groups of normally distributed observations to find out whether all groups have the same mean or not. + +**Syntax** + +```sql +analysisOfVariance(val, group_no) +``` + +Aliases: `anova` + +**Parameters** +- `val`: value. +- `group_no` : group number that `val` belongs to. + +:::note +Groups are enumerated starting from 0 and there should be at least two groups to perform a test. +Moreover there should be at least one group with the number of observations greater than one. +::: + +**Returned value** + +- `(f_statistic, p_value)`. [Tuple](../../data-types/tuple.md)([Float64](../../data-types/float.md), [Float64](../../data-types/float.md)) + +**Implementation details** + + +**Example** + +Query: + +```sql +``` + +Result: + +```response +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index e9a7fe4fc2b..451ee2aae9d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -37,6 +37,7 @@ Standard aggregate functions: ClickHouse-specific aggregate functions: +- [analysisOfVariance](/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md) - [any](/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md) - [anyHeavy](/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md) - [anyLast](/docs/en/sql-reference/aggregate-functions/reference/anylast.md) From 0ce90f4ef44a8c3f51e3bb57e810c5bef6e1f492 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Sun, 12 May 2024 18:13:17 +0300 Subject: [PATCH 17/85] Prevent conversion to Replicated if zookeeper path already exists --- src/Databases/DatabaseOrdinary.cpp | 14 ++++++ .../configs/config.d/clusters.xml | 4 +- .../test_modify_engine_on_restart/test.py | 45 +++++++++++++++++-- 3 files changed, 59 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index d571fe4491d..22b553ecd1e 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -76,6 +76,20 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex String replica_path = server_settings.default_replica_path; String replica_name = server_settings.default_replica_name; + /// Check that replica path doesn't exist + Macros::MacroExpansionInfo info; + StorageID table_id = StorageID(create_query->getDatabase(), create_query->getTable(), create_query->uuid); + info.table_id = table_id; + info.expand_special_macros_only = false; + + String zookeeper_path = context->getMacros()->expand(replica_path, info); + if (context->getZooKeeper()->exists(zookeeper_path)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Found existing ZooKeeper path {} while trying to convert table {} to replicated. Table will not be converted.", + zookeeper_path, backQuote(table_id.getFullTableName()) + ); + auto args = std::make_shared(); args->children.push_back(std::make_shared(replica_path)); args->children.push_back(std::make_shared(replica_name)); diff --git a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml index d3a9d4fb8f0..fbcf6499ec1 100644 --- a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml +++ b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml @@ -15,8 +15,10 @@ +/clickhouse/tables/{database}/{table}/{uuid} + 01 - \ No newline at end of file + diff --git a/tests/integration/test_modify_engine_on_restart/test.py b/tests/integration/test_modify_engine_on_restart/test.py index 289b25dd89e..8af7ec8ae83 100644 --- a/tests/integration/test_modify_engine_on_restart/test.py +++ b/tests/integration/test_modify_engine_on_restart/test.py @@ -1,5 +1,9 @@ import pytest -from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags +from test_modify_engine_on_restart.common import ( + check_flags_deleted, + set_convert_flags, + get_table_path, +) from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -120,7 +124,7 @@ def check_replica_added(): q( ch2, - f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{uuid}/{{shard}}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A", + f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/rmt/{uuid}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A", ) ch2.query(database=database_name, sql="SYSTEM SYNC REPLICA rmt", timeout=20) @@ -136,7 +140,7 @@ def check_replica_added(): def test_modify_engine_on_restart(started_cluster): - ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") + ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster") create_tables() @@ -159,3 +163,38 @@ def test_modify_engine_on_restart(started_cluster): ch1.restart_clickhouse() check_tables(True) + + +def test_modify_engine_fails_if_zk_path_exists(started_cluster): + ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster") + + q( + ch1, + "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + uuid = q( + ch1, + f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", + ).strip("'[]\n") + + q( + ch1, + f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + + set_convert_flags(ch1, database_name, ["already_exists_1"]) + + table_data_path = get_table_path(ch1, "already_exists_1", database_name) + + ch1.stop_clickhouse() + ch1.start_clickhouse(retry_start=False, expected_to_fail=True) + + # Check if we can cancel convertation + ch1.exec_in_container( + [ + "bash", + "-c", + f"rm {table_data_path}convert_to_replicated", + ] + ) + ch1.start_clickhouse() From 5f2228cd16dcdfc212450c214ef4014410e1b0cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 13 May 2024 00:35:38 +0300 Subject: [PATCH 18/85] Fix test --- .../integration/test_modify_engine_on_restart/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_modify_engine_on_restart/test.py b/tests/integration/test_modify_engine_on_restart/test.py index 8af7ec8ae83..81854340752 100644 --- a/tests/integration/test_modify_engine_on_restart/test.py +++ b/tests/integration/test_modify_engine_on_restart/test.py @@ -40,8 +40,8 @@ def started_cluster(): cluster.shutdown() -def q(node, query): - return node.query(database=database_name, sql=query) +def q(node, query, database=database_name): + return node.query(database=database, sql=query) def create_tables(): @@ -166,20 +166,24 @@ def test_modify_engine_on_restart(started_cluster): def test_modify_engine_fails_if_zk_path_exists(started_cluster): - ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster") + database_name = "zk_path" + ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") q( ch1, "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + database_name, ) uuid = q( ch1, f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", + database_name, ).strip("'[]\n") q( ch1, f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;", + database_name, ) set_convert_flags(ch1, database_name, ["already_exists_1"]) From 7543cd372c555a660c0d8149c68b45ab5f9bd7f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 13 May 2024 18:52:25 +0300 Subject: [PATCH 19/85] Move test to another file --- .../configs/config.d/clusters.xml | 2 - ...sters_unusual.xml => clusters_zk_path.xml} | 2 +- .../test_modify_engine_on_restart/test.py | 53 ++---------------- .../{test_unusual_path.py => test_zk_path.py} | 55 +++++++++++++++++-- 4 files changed, 55 insertions(+), 57 deletions(-) rename tests/integration/test_modify_engine_on_restart/configs/config.d/{clusters_unusual.xml => clusters_zk_path.xml} (80%) rename tests/integration/test_modify_engine_on_restart/{test_unusual_path.py => test_zk_path.py} (52%) diff --git a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml index fbcf6499ec1..c8bbb7f3530 100644 --- a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml +++ b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters.xml @@ -15,8 +15,6 @@ -/clickhouse/tables/{database}/{table}/{uuid} - 01 diff --git a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_unusual.xml b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_zk_path.xml similarity index 80% rename from tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_unusual.xml rename to tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_zk_path.xml index 812291335b8..ba13cd87031 100644 --- a/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_unusual.xml +++ b/tests/integration/test_modify_engine_on_restart/configs/config.d/clusters_zk_path.xml @@ -15,6 +15,6 @@ 01 -/lol/kek/'/{uuid} +/clickhouse/'/{database}/{table}/{uuid} diff --git a/tests/integration/test_modify_engine_on_restart/test.py b/tests/integration/test_modify_engine_on_restart/test.py index 81854340752..289b25dd89e 100644 --- a/tests/integration/test_modify_engine_on_restart/test.py +++ b/tests/integration/test_modify_engine_on_restart/test.py @@ -1,9 +1,5 @@ import pytest -from test_modify_engine_on_restart.common import ( - check_flags_deleted, - set_convert_flags, - get_table_path, -) +from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -40,8 +36,8 @@ def started_cluster(): cluster.shutdown() -def q(node, query, database=database_name): - return node.query(database=database, sql=query) +def q(node, query): + return node.query(database=database_name, sql=query) def create_tables(): @@ -124,7 +120,7 @@ def check_replica_added(): q( ch2, - f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/rmt/{uuid}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A", + f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{uuid}/{{shard}}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A", ) ch2.query(database=database_name, sql="SYSTEM SYNC REPLICA rmt", timeout=20) @@ -140,7 +136,7 @@ def check_replica_added(): def test_modify_engine_on_restart(started_cluster): - ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster") + ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") create_tables() @@ -163,42 +159,3 @@ def test_modify_engine_on_restart(started_cluster): ch1.restart_clickhouse() check_tables(True) - - -def test_modify_engine_fails_if_zk_path_exists(started_cluster): - database_name = "zk_path" - ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") - - q( - ch1, - "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", - database_name, - ) - uuid = q( - ch1, - f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", - database_name, - ).strip("'[]\n") - - q( - ch1, - f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;", - database_name, - ) - - set_convert_flags(ch1, database_name, ["already_exists_1"]) - - table_data_path = get_table_path(ch1, "already_exists_1", database_name) - - ch1.stop_clickhouse() - ch1.start_clickhouse(retry_start=False, expected_to_fail=True) - - # Check if we can cancel convertation - ch1.exec_in_container( - [ - "bash", - "-c", - f"rm {table_data_path}convert_to_replicated", - ] - ) - ch1.start_clickhouse() diff --git a/tests/integration/test_modify_engine_on_restart/test_unusual_path.py b/tests/integration/test_modify_engine_on_restart/test_zk_path.py similarity index 52% rename from tests/integration/test_modify_engine_on_restart/test_unusual_path.py rename to tests/integration/test_modify_engine_on_restart/test_zk_path.py index e82f48e8b34..8bbfe64240f 100644 --- a/tests/integration/test_modify_engine_on_restart/test_unusual_path.py +++ b/tests/integration/test_modify_engine_on_restart/test_zk_path.py @@ -1,12 +1,16 @@ import pytest -from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags +from test_modify_engine_on_restart.common import ( + check_flags_deleted, + get_table_path, + set_convert_flags, +) from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) ch1 = cluster.add_instance( "ch1", main_configs=[ - "configs/config.d/clusters_unusual.xml", + "configs/config.d/clusters_zk_path.xml", "configs/config.d/distributed_ddl.xml", ], with_zookeeper=True, @@ -27,8 +31,8 @@ def started_cluster(): cluster.shutdown() -def q(node, query): - return node.query(database=database_name, sql=query) +def q(node, query, database=database_name): + return node.query(database=database, sql=query) def create_tables(): @@ -63,7 +67,7 @@ def check_tables(): ) .strip() .startswith( - "ReplicatedReplacingMergeTree(\\'/lol/kek/\\\\\\'/{uuid}\\', \\'{replica}\\', D)" + "ReplicatedReplacingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', D)" ) ) assert ( @@ -73,7 +77,7 @@ def check_tables(): ) .strip() .startswith( - "ReplicatedVersionedCollapsingMergeTree(\\'/lol/kek/\\\\\\'/{uuid}\\', \\'{replica}\\', Sign, Version)" + "ReplicatedVersionedCollapsingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', Sign, Version)" ) ) @@ -90,3 +94,42 @@ def test_modify_engine_on_restart_with_unusual_path(started_cluster): check_flags_deleted(ch1, database_name, ["replacing_ver", "collapsing_ver"]) check_tables() + + +def test_modify_engine_fails_if_zk_path_exists(started_cluster): + database_name = "zk_path" + ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") + + q( + ch1, + "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + database_name, + ) + uuid = q( + ch1, + f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", + database_name, + ).strip("'[]\n") + + q( + ch1, + f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/\\'/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;", + database_name, + ) + + set_convert_flags(ch1, database_name, ["already_exists_1"]) + + table_data_path = get_table_path(ch1, "already_exists_1", database_name) + + ch1.stop_clickhouse() + ch1.start_clickhouse(retry_start=False, expected_to_fail=True) + + # Check if we can cancel convertation + ch1.exec_in_container( + [ + "bash", + "-c", + f"rm {table_data_path}convert_to_replicated", + ] + ) + ch1.start_clickhouse() From 54e9ffd0bf7d9706aabd3690f95ff81bcd5f8eb9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Wed, 15 May 2024 02:31:19 +0300 Subject: [PATCH 20/85] Move test to new file --- .../test_unusual_path.py | 92 +++++++++++++++++++ .../test_zk_path.py | 74 +-------------- 2 files changed, 96 insertions(+), 70 deletions(-) create mode 100644 tests/integration/test_modify_engine_on_restart/test_unusual_path.py diff --git a/tests/integration/test_modify_engine_on_restart/test_unusual_path.py b/tests/integration/test_modify_engine_on_restart/test_unusual_path.py new file mode 100644 index 00000000000..20d2c29257b --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/test_unusual_path.py @@ -0,0 +1,92 @@ +import pytest +from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance( + "ch1", + main_configs=[ + "configs/config.d/clusters_zk_path.xml", + "configs/config.d/distributed_ddl.xml", + ], + with_zookeeper=True, + macros={"replica": "node1"}, + stay_alive=True, +) + +database_name = "modify_engine_unusual_path" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def q(node, query): + return node.query(database=database_name, sql=query) + + +def create_tables(): + # Check one argument + q( + ch1, + "CREATE TABLE replacing_ver ( A Int64, D Date, S String ) ENGINE = ReplacingMergeTree(D) PARTITION BY toYYYYMM(D) ORDER BY A", + ) + + # Check more than one argument + q( + ch1, + "CREATE TABLE collapsing_ver ( ID UInt64, Sign Int8, Version UInt8 ) ENGINE = VersionedCollapsingMergeTree(Sign, Version) ORDER BY ID", + ) + + +def check_tables(): + # Check tables exists + assert ( + q( + ch1, + "SHOW TABLES", + ).strip() + == "collapsing_ver\nreplacing_ver" + ) + + # Check engines + assert ( + q( + ch1, + f"SELECT engine_full FROM system.tables WHERE database = '{database_name}' and name = 'replacing_ver'", + ) + .strip() + .startswith( + "ReplicatedReplacingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', D)" + ) + ) + assert ( + q( + ch1, + f"SELECT engine_full FROM system.tables WHERE database = '{database_name}' and name = 'collapsing_ver'", + ) + .strip() + .startswith( + "ReplicatedVersionedCollapsingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', Sign, Version)" + ) + ) + + +def test_modify_engine_on_restart_with_unusual_path(started_cluster): + ch1.query("CREATE DATABASE " + database_name) + + create_tables() + + set_convert_flags(ch1, database_name, ["replacing_ver", "collapsing_ver"]) + + ch1.restart_clickhouse() + + check_flags_deleted(ch1, database_name, ["replacing_ver", "collapsing_ver"]) + + check_tables() diff --git a/tests/integration/test_modify_engine_on_restart/test_zk_path.py b/tests/integration/test_modify_engine_on_restart/test_zk_path.py index 8bbfe64240f..dd633ad0810 100644 --- a/tests/integration/test_modify_engine_on_restart/test_zk_path.py +++ b/tests/integration/test_modify_engine_on_restart/test_zk_path.py @@ -1,6 +1,5 @@ import pytest from test_modify_engine_on_restart.common import ( - check_flags_deleted, get_table_path, set_convert_flags, ) @@ -18,7 +17,7 @@ ch1 = cluster.add_instance( stay_alive=True, ) -database_name = "modify_engine_unusual_path" +database_name = "modify_engine_zk_path" @pytest.fixture(scope="module") @@ -31,90 +30,25 @@ def started_cluster(): cluster.shutdown() -def q(node, query, database=database_name): - return node.query(database=database, sql=query) - - -def create_tables(): - # Check one argument - q( - ch1, - "CREATE TABLE replacing_ver ( A Int64, D Date, S String ) ENGINE = ReplacingMergeTree(D) PARTITION BY toYYYYMM(D) ORDER BY A", - ) - - # Check more than one argument - q( - ch1, - "CREATE TABLE collapsing_ver ( ID UInt64, Sign Int8, Version UInt8 ) ENGINE = VersionedCollapsingMergeTree(Sign, Version) ORDER BY ID", - ) - - -def check_tables(): - # Check tables exists - assert ( - q( - ch1, - "SHOW TABLES", - ).strip() - == "collapsing_ver\nreplacing_ver" - ) - - # Check engines - assert ( - q( - ch1, - f"SELECT engine_full FROM system.tables WHERE database = '{database_name}' and name = 'replacing_ver'", - ) - .strip() - .startswith( - "ReplicatedReplacingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', D)" - ) - ) - assert ( - q( - ch1, - f"SELECT engine_full FROM system.tables WHERE database = '{database_name}' and name = 'collapsing_ver'", - ) - .strip() - .startswith( - "ReplicatedVersionedCollapsingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', Sign, Version)" - ) - ) - - -def test_modify_engine_on_restart_with_unusual_path(started_cluster): - ch1.query("CREATE DATABASE " + database_name) - - create_tables() - - set_convert_flags(ch1, database_name, ["replacing_ver", "collapsing_ver"]) - - ch1.restart_clickhouse() - - check_flags_deleted(ch1, database_name, ["replacing_ver", "collapsing_ver"]) - - check_tables() +def q(node, query): + return node.query(database=database_name, sql=query) def test_modify_engine_fails_if_zk_path_exists(started_cluster): - database_name = "zk_path" - ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster") + ch1.query("CREATE DATABASE " + database_name) q( ch1, "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", - database_name, ) uuid = q( ch1, f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", - database_name, ).strip("'[]\n") q( ch1, f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/\\'/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;", - database_name, ) set_convert_flags(ch1, database_name, ["already_exists_1"]) From 13fc7c7cf81c806f5993ca43c4114e4073eb2821 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 May 2024 10:35:10 +0200 Subject: [PATCH 21/85] Don't allow 0 for max_block_size --- src/Core/SettingsQuirks.cpp | 16 ++++++++++++---- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 16 ++++------------ .../03149_numbers_max_block_size_zero.reference | 3 +-- .../03149_numbers_max_block_size_zero.sh | 7 +++++++ .../03149_numbers_max_block_size_zero.sql | 2 -- 5 files changed, 24 insertions(+), 20 deletions(-) create mode 100755 tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh delete mode 100644 tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5e7d02dc448..73a0e2a9a6c 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -92,7 +92,7 @@ void applySettingsQuirks(Settings & settings, LoggerPtr log) void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) { - auto getCurrentValue = [¤t_settings](const std::string_view name) -> Field + auto get_current_value = [¤t_settings](const std::string_view name) -> Field { Field current_value; bool has_current_value = current_settings.tryGet(name, current_value); @@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) return current_value; }; - UInt64 max_threads = getCurrentValue("max_threads").get(); + UInt64 max_threads = get_current_value("max_threads").get(); UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores(); if (max_threads > max_threads_max_value) { @@ -109,7 +109,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) current_settings.set("max_threads", max_threads_max_value); } - constexpr UInt64 max_sane_block_rows_size = 4294967296; // 2^32 + static constexpr UInt64 max_sane_block_rows_size = 4294967296; // 2^32 std::unordered_set block_rows_settings{ "max_block_size", "max_insert_block_size", @@ -120,7 +120,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "input_format_parquet_max_block_size"}; for (auto const & setting : block_rows_settings) { - auto block_size = getCurrentValue(setting).get(); + auto block_size = get_current_value(setting).get(); if (block_size > max_sane_block_rows_size) { if (log) @@ -128,5 +128,13 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) current_settings.set(setting, max_sane_block_rows_size); } } + + if (auto max_block_size = get_current_value("max_block_size").get(); max_block_size == 0) + { + if (log) + LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE); + current_settings.set("max_block_size", DEFAULT_BLOCK_SIZE); + } } + } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 759dc7354df..11371578c79 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -176,9 +176,8 @@ protected: { std::lock_guard lock(ranges_state->mutex); - UInt64 need = base_block_size_; - bool without_block_size_limit = need == 0; + UInt64 need = base_block_size_; UInt64 size = 0; /// how many item found. /// find start @@ -186,21 +185,14 @@ protected: end = start; /// find end - while (without_block_size_limit || need != 0) + while (need != 0) { UInt128 can_provide = end.offset_in_ranges == ranges.size() ? static_cast(0) : ranges[end.offset_in_ranges].size - end.offset_in_range; - if (can_provide == 0) break; - if (without_block_size_limit) - { - end.offset_in_ranges++; - end.offset_in_range = 0; - size += static_cast(can_provide); - } - else if (can_provide > need) + if (can_provide > need) { end.offset_in_range += need; size += need; @@ -535,7 +527,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() checkLimits(size_t(total_size)); - if (max_block_size != 0 && total_size / max_block_size < num_streams) + if (total_size / max_block_size < num_streams) num_streams = static_cast(total_size / max_block_size); if (num_streams == 0) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference index 896f02d1185..d86bac9de59 100644 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.reference @@ -1,2 +1 @@ -1320 -1320 +OK diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh new file mode 100755 index 00000000000..6f70a0d2536 --- /dev/null +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | grep -q "Sanity check: 'max_block_size' cannot be 0. Set to default value" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql deleted file mode 100644 index afc4e4d57a5..00000000000 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sql +++ /dev/null @@ -1,2 +0,0 @@ -SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0; -SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 1; From 4483da06e819e902c273d6e84d3d58971d73c30c Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 15:42:40 +0200 Subject: [PATCH 22/85] Update rowNumberInXYZ functions --- .../functions/other-functions.md | 105 +++++++++++++++++- 1 file changed, 99 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2b0215115cb..7c930d23919 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -998,17 +998,110 @@ SELECT version() Returns the build ID generated by a compiler for the running ClickHouse server binary. If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. -## blockNumber() +## blockNumber -Returns the sequence number of the data block where the row is located. +Returns the sequence number of the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks) where the row is located. -## rowNumberInBlock() {#rowNumberInBlock} +**Syntax** + +```sql +blockNumber() +``` + +**Returned value** + +- Sequence number of the data block where the row is located. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT groupUniqArray(blockNumber()) +FROM +( + SELECT * + FROM system.numbers_mt + LIMIT 100000 +) +``` + +Result: + +```response +┌─groupUniqArray(blockNumber())─┐ +│ [6,7] │ +└───────────────────────────────┘ +``` + +## rowNumberInBlock {#rowNumberInBlock} + +Returns the ordinal number of the row in the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks). Different data blocks are always recalculated. + +**Syntax** + +```sql +rowNumberInBlock() +``` + +**Returned value** + +- Ordinal number of the row in the data block starting from 0. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + min(rowNumberInBlock()), + max(rowNumberInBlock()) +FROM (SELECT * FROM system.numbers_mt LIMIT 100000); +``` + +Result: + +```response +┌─min(rowNumberInBlock())─┬─max(rowNumberInBlock())─┐ +│ 0 │ 65408 │ +└─────────────────────────┴─────────────────────────┘ +``` + +## rowNumberInAllBlocks + +Returns the ordinal number of the row in the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks). This function only considers the affected data blocks. + +**Syntax** + +```sql +rowNumberInAllBlocks() +``` + +**Returned value** + +- Ordinal number of the row in the data block starting from 0. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + min(rowNumberInAllBlocks()), + max(rowNumberInAllBlocks()) +FROM (SELECT * FROM system.numbers_mt LIMIT 100000); +``` + +Result: + +```response +┌─min(rowNumberInAllBlocks())─┬─max(rowNumberInAllBlocks())─┐ +│ 0 │ 99999 │ +└─────────────────────────────┴─────────────────────────────┘ +``` -Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. -## rowNumberInAllBlocks() -Returns the ordinal number of the row in the data block. This function only considers the affected data blocks. ## neighbor From 1be205e3f4528b6d09775dee5560b26ee99db67b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 16:22:25 +0200 Subject: [PATCH 23/85] Update anova documentation --- .../reference/analysis_of_variance.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md b/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md index 7f0df74010b..d9b44b3ff07 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md +++ b/docs/en/sql-reference/aggregate-functions/reference/analysis_of_variance.md @@ -21,24 +21,25 @@ Aliases: `anova` :::note Groups are enumerated starting from 0 and there should be at least two groups to perform a test. -Moreover there should be at least one group with the number of observations greater than one. +There should be at least one group with the number of observations greater than one. ::: **Returned value** -- `(f_statistic, p_value)`. [Tuple](../../data-types/tuple.md)([Float64](../../data-types/float.md), [Float64](../../data-types/float.md)) - -**Implementation details** - +- `(f_statistic, p_value)`. [Tuple](../../data-types/tuple.md)([Float64](../../data-types/float.md), [Float64](../../data-types/float.md)). **Example** Query: ```sql +SELECT analysisOfVariance(number, number % 2) FROM numbers(1048575); ``` Result: ```response +┌─analysisOfVariance(number, modulo(number, 2))─┐ +│ (0,1) │ +└───────────────────────────────────────────────┘ ``` From ea52b25dcb451b5059cf9e4c9136c2f328340b90 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 18:49:45 +0200 Subject: [PATCH 24/85] 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 a69ca0fb644..8f43c533fce 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -53,6 +53,8 @@ AutoFDO AutoML Autocompletion AvroConfluent +analysisOfVariance +ANOVA BIGINT BIGSERIAL BORO From fd322df568d115afe9d2d9564aa7d64f8b3bc114 Mon Sep 17 00:00:00 2001 From: Yong Wang Date: Thu, 16 May 2024 10:24:56 -0700 Subject: [PATCH 25/85] fix_client_abort: Resolve Client Abortion Issue When Using KQL Table Function in Interactive Mode --- src/Parsers/Kusto/ParserKQLStatement.cpp | 83 ++++++++++++------------ 1 file changed, 43 insertions(+), 40 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index fbf2110e664..e508b69bdff 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -62,49 +63,51 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ParserKQLWithUnionQuery kql_p; - ASTPtr select; - ParserToken s_lparen(TokenType::OpeningRoundBracket); + ParserToken lparen(TokenType::OpeningRoundBracket); - auto begin = pos; - auto paren_count = 0; + ASTPtr string_literal; + ParserStringLiteral parser_string_literal; + + if (!lparen.ignore(pos, expected)) + return false; + + size_t paren_count = 0; String kql_statement; - - if (s_lparen.ignore(pos, expected)) + if (parser_string_literal.parse(pos, string_literal, expected)) { - if (pos->type == TokenType::HereDoc) - { - kql_statement = String(pos->begin + 2, pos->end - 2); - } - else - { - ++paren_count; - auto pos_start = pos; - while (isValidKQLPos(pos)) - { - if (pos->type == TokenType::ClosingRoundBracket) - --paren_count; - if (pos->type == TokenType::OpeningRoundBracket) - ++paren_count; - - if (paren_count == 0) - break; - ++pos; - } - kql_statement = String(pos_start->begin, (--pos)->end); - } - ++pos; - Tokens token_kql(kql_statement.c_str(), kql_statement.c_str() + kql_statement.size()); - IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); - - if (kql_p.parse(pos_kql, select, expected)) - { - node = select; - ++pos; - return true; - } + kql_statement = typeid_cast(*string_literal).value.safeGet(); } - pos = begin; - return false; + else + { + ++paren_count; + auto pos_start = pos; + while (isValidKQLPos(pos)) + { + if (pos->type == TokenType::ClosingRoundBracket) + --paren_count; + if (pos->type == TokenType::OpeningRoundBracket) + ++paren_count; + + if (paren_count == 0) + break; + ++pos; + } + if (!isValidKQLPos(pos)) + { + return false; + } + --pos; + kql_statement = String(pos_start->begin, pos->end); + ++pos; + } + + Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size()); + IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); + Expected kql_expected; + kql_expected.enable_highlighting = false; + if (!ParserKQLWithUnionQuery().parse(pos_kql, node, kql_expected)) + return false; + ++pos; + return true; } } From cd73164005de04b1c198ec810ed75926c72f9074 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 May 2024 21:08:29 +0000 Subject: [PATCH 26/85] slightly better setting force_optimize_projection_name --- src/Interpreters/executeQuery.cpp | 28 --------------- .../QueryPlan/Optimizations/Optimizations.h | 7 ++-- .../QueryPlanOptimizationSettings.cpp | 2 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 24 ++++++++----- .../optimizeUseAggregateProjection.cpp | 35 ++++++------------- .../optimizeUseNormalProjection.cpp | 19 +++++----- ...6_force_optimize_projection_name.reference | 2 ++ .../02906_force_optimize_projection_name.sql | 15 ++++++++ 8 files changed, 59 insertions(+), 73 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 33a4cf2a74c..67797f7d4f6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1254,34 +1254,6 @@ static std::tuple executeQueryImpl( } } } - // Here we check if our our projections contain force_optimize_projection_name - if (!settings.force_optimize_projection_name.value.empty()) - { - bool found = false; - std::set projections; - { - const auto & access_info = context->getQueryAccessInfo(); - std::lock_guard lock(access_info.mutex); - projections = access_info.projections; - } - - for (const auto &projection : projections) - { - // projection value has structure like: .. - // We need to get only the projection name - size_t last_dot_pos = projection.find_last_of('.'); - std::string projection_name = (last_dot_pos != std::string::npos) ? projection.substr(last_dot_pos + 1) : projection; - if (settings.force_optimize_projection_name.value == projection_name) - { - found = true; - break; - } - } - - if (!found) - throw Exception(ErrorCodes::INCORRECT_DATA, "Projection {} is specified in setting force_optimize_projection_name but not used", - settings.force_optimize_projection_name.value); - } if (process_list_entry) { diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 18f1496d26a..b33a373a970 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -111,8 +111,11 @@ void optimizePrimaryKeyCondition(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); -bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); + +/// Returns the name of used projection or nullopt if no projection is used. +std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); +std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); + bool addPlansForSets(QueryPlan & plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 80923159ddc..2738de1ff5f 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -46,7 +46,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.optimize_projection = from.optimize_use_projections; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; - settings.force_projection_name = from.force_optimize_projection_name; + settings.force_projection_name = settings.optimize_projection ? from.force_optimize_projection_name.value : ""; settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections; return settings; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 915e664ea8f..4f512016c6b 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -106,7 +106,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { const size_t max_optimizations_to_apply = optimization_settings.max_optimizations_to_apply; - size_t num_applied_projection = 0; + std::unordered_set applied_projection_names; bool has_reading_from_mt = false; Stack stack; @@ -159,9 +159,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s /// Projection optimization relies on PK optimization if (optimization_settings.optimize_projection) - num_applied_projection - += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); - + { + auto applied_projection = optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); + if (applied_projection) + applied_projection_names.insert(*applied_projection); + } if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); @@ -180,11 +182,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.optimize_projection) { /// Projection optimization relies on PK optimization - if (optimizeUseNormalProjections(stack, nodes)) + if (auto applied_projection = optimizeUseNormalProjections(stack, nodes)) { - ++num_applied_projection; + applied_projection_names.insert(*applied_projection); - if (max_optimizations_to_apply && max_optimizations_to_apply < num_applied_projection) + if (max_optimizations_to_apply && max_optimizations_to_apply < applied_projection_names.size()) throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS, "Too many projection optimizations applied to query plan. Current limit {}", max_optimizations_to_apply); @@ -201,10 +203,16 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s stack.pop_back(); } - if (optimization_settings.force_use_projection && has_reading_from_mt && num_applied_projection == 0) + if (optimization_settings.force_use_projection && has_reading_from_mt && applied_projection_names.empty()) throw Exception( ErrorCodes::PROJECTION_NOT_USED, "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); + + if (!optimization_settings.force_projection_name.empty() && has_reading_from_mt && !applied_projection_names.contains(optimization_settings.force_projection_name)) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Projection {} is specified in setting force_optimize_projection_name but not used", + optimization_settings.force_projection_name); } void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 30ff9970790..4017670ad14 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -552,28 +552,28 @@ static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections) +std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections) { if (node.children.size() != 1) - return false; + return {}; auto * aggregating = typeid_cast(node.step.get()); if (!aggregating) - return false; + return {}; if (!aggregating->canUseProjection()) - return false; + return {}; QueryPlan::Node * reading_node = findReadingStep(*node.children.front()); if (!reading_node) - return false; + return {}; auto * reading = typeid_cast(reading_node->step.get()); if (!reading) - return false; + return {}; if (!canUseProjectionForReadingStep(reading)) - return false; + return {}; std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); @@ -597,7 +597,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & if (ordinary_reading_marks == 0) { reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return false; + return {}; } const auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges; @@ -631,15 +631,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & if (!best_candidate) { reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return false; + return {}; } } else { - return false; + return {}; } - Context::QualifiedProjectionName projection_name; chassert(best_candidate != nullptr); QueryPlanStepPtr projection_reading; @@ -654,12 +653,6 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); projection_reading = std::make_unique(std::move(pipe)); has_ordinary_parts = false; - - projection_name = Context::QualifiedProjectionName - { - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = candidates.minmax_projection->candidate.projection->name, - }; } else { @@ -691,12 +684,6 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique(std::move(pipe)); } - 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; if (has_ordinary_parts) reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); @@ -746,7 +733,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & node.children.push_back(&expr_or_filter_node); } - return true; + return best_candidate->projection->name; } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 13c6c6b0821..728aaaa6fc4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -73,16 +73,16 @@ static bool hasAllRequiredColumns(const ProjectionDescription * projection, cons } -bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) +std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { const auto & frame = stack.back(); auto * reading = typeid_cast(frame.node->step.get()); if (!reading) - return false; + return {}; if (!canUseProjectionForReadingStep(reading)) - return false; + return {}; auto iter = stack.rbegin(); while (std::next(iter) != stack.rend()) @@ -96,7 +96,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) /// Dangling query plan node. This might be generated by StorageMerge. if (iter->node->step.get() == reading) - return false; + return {}; const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -107,7 +107,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) normal_projections.push_back(&projection); if (normal_projections.empty()) - return false; + return {}; ContextPtr context = reading->getContext(); auto it = std::find_if(normal_projections.begin(), normal_projections.end(), [&](const auto * projection) @@ -126,7 +126,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) { auto & child = iter->node->children[iter->next_child - 1]; if (!query.build(*child)) - return false; + return {}; if (query.dag) query.dag->removeUnusedActions(); @@ -146,7 +146,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (ordinary_reading_marks == 0) { reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return false; + return {}; } const auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges; @@ -185,7 +185,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!best_candidate) { reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return false; + return {}; } auto storage_snapshot = reading->getStorageSnapshot(); @@ -283,8 +283,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) /// Here we remove last steps from stack to be able to optimize again. /// In theory, read-in-order can be applied to projection. stack.resize(iter.base() - stack.begin()); - - return true; + return best_candidate->projection->name; } } diff --git a/tests/queries/0_stateless/02906_force_optimize_projection_name.reference b/tests/queries/0_stateless/02906_force_optimize_projection_name.reference index 9daeafb9864..679eff3f0b4 100644 --- a/tests/queries/0_stateless/02906_force_optimize_projection_name.reference +++ b/tests/queries/0_stateless/02906_force_optimize_projection_name.reference @@ -1 +1,3 @@ test +1 +0 diff --git a/tests/queries/0_stateless/02906_force_optimize_projection_name.sql b/tests/queries/0_stateless/02906_force_optimize_projection_name.sql index 952ef8178b7..6b9d7f74f9f 100644 --- a/tests/queries/0_stateless/02906_force_optimize_projection_name.sql +++ b/tests/queries/0_stateless/02906_force_optimize_projection_name.sql @@ -1,3 +1,5 @@ +DROP TABLE IF EXISTS test; + CREATE TABLE test ( `id` UInt64, @@ -18,3 +20,16 @@ SELECT name FROM test GROUP BY name SETTINGS force_optimize_projection_name='pro SELECT name FROM test GROUP BY name SETTINGS force_optimize_projection_name='non_existing_projection'; -- { serverError 117 } SELECT name FROM test SETTINGS force_optimize_projection_name='projection_name'; -- { serverError 117 } + +INSERT INTO test SELECT number, 'test' FROM numbers(1, 100) SETTINGS force_optimize_projection_name='projection_name'; +SELECT 1 SETTINGS force_optimize_projection_name='projection_name'; + +SYSTEM FLUSH LOGS; + +SELECT read_rows FROM system.query_log +WHERE current_database = currentDatabase() + AND query LIKE '%SELECT name FROM test%' + AND Settings['force_optimize_projection_name'] = 'projection_name' + AND type = 'ExceptionBeforeStart'; + +DROP TABLE test; From 04f631dec9743da6655586073c28c895cfd7cd76 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 May 2024 10:13:13 +0200 Subject: [PATCH 27/85] Better --- src/Core/SettingsQuirks.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 73a0e2a9a6c..5541cc19653 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -120,8 +120,8 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "input_format_parquet_max_block_size"}; for (auto const & setting : block_rows_settings) { - auto block_size = get_current_value(setting).get(); - if (block_size > max_sane_block_rows_size) + if (auto block_size = get_current_value(setting).get(); + block_size > max_sane_block_rows_size) { if (log) LOG_WARNING(log, "Sanity check: '{}' value is too high ({}). Reduced to {}", setting, block_size, max_sane_block_rows_size); From d3c9b5f52194f58cf06d829bceb0bb4e3cbe3c62 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 17 May 2024 13:08:50 +0200 Subject: [PATCH 28/85] Fix error codes --- src/Interpreters/executeQuery.cpp | 1 - src/Processors/QueryPlan/Optimizations/optimizeTree.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 67797f7d4f6..e603e2624aa 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -103,7 +103,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int QUERY_WAS_CANCELLED; - extern const int INCORRECT_DATA; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; extern const int INCORRECT_QUERY; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 4f512016c6b..df9e095af30 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { + extern const int INCORRECT_DATA; extern const int TOO_MANY_QUERY_PLAN_OPTIMIZATIONS; extern const int PROJECTION_NOT_USED; } From 226de66d3603e65aa10e6b687642cd6a89edeb7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 17 May 2024 13:33:14 +0000 Subject: [PATCH 29/85] Add missing settings to recoverLostReplica --- src/Databases/DatabaseReplicated.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7b1c49c2cf8..78d502ec2c7 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -944,6 +944,13 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_hyperscan", 1); query_context->setSetting("allow_simdjson", 1); query_context->setSetting("allow_deprecated_syntax_for_merge_tree", 1); + query_context->setSetting("allow_suspicious_primary_key", 1); + query_context->setSetting("allow_suspicious_ttl_expressions", 1); + query_context->setSetting("allow_suspicious_variant_types", 1); + query_context->setSetting("enable_deflate_qpl_codec", 1); + query_context->setSetting("enable_zstd_qat_codec", 1); + query_context->setSetting("allow_create_index_without_type", 1); + query_context->setSetting("allow_experimental_s3queue", 1); auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); query_context->initZooKeeperMetadataTransaction(txn); From e3329a10f92f429d70ba19ba9bf72e0279d43668 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 18 May 2024 18:00:32 +0200 Subject: [PATCH 30/85] CI: mergeable check redesign --- .github/workflows/pull_request.yml | 27 ++++++++++++++++++++++++++- tests/ci/ci_config.py | 4 +++- tests/ci/commit_status_helper.py | 9 ++++++++- tests/ci/merge_pr.py | 19 ++++++++++++++++++- 4 files changed, 55 insertions(+), 4 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9f16e32707e..e0eda476d19 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -130,6 +130,14 @@ jobs: with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} + # stage for jobs that do not prohibit merge + Tests_3: + needs: [RunConfig, Tests_1, Tests_2] + if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} + uses: ./.github/workflows/reusable_test_stage.yml + with: + stage: Tests_3 + data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# # Reports should by run even if Builds_1/2 fail, so put them separatly in wf (not in Tests_1/2) @@ -156,15 +164,32 @@ jobs: runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} + CheckReadyForMerge: + if: ${{ !failure() && !cancelled() }} + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] + runs-on: [self-hosted, style-checker-aarch64] + steps: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + filter: tree:0 + - name: Check and set merge status + if: ${{ github.event_name != 'merge_group' }} + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 merge_pr.py --set-status + ################################# Stage Final ################################# # FinishCheck: if: ${{ !failure() && !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3, CheckReadyForMerge] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code uses: ClickHouse/checkout@v1 + with: + filter: tree:0 - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 588f4934125..6d450a79a69 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -25,6 +25,7 @@ class CIStages(metaclass=WithIter): BUILDS_2 = "Builds_2" TESTS_1 = "Tests_1" TESTS_2 = "Tests_2" + TESTS_3 = "Tests_3" class Runners(metaclass=WithIter): @@ -579,7 +580,6 @@ class CIConfig: elif job_name == JobNames.BUILD_CHECK_SPECIAL: stage_type = CIStages.TESTS_2 elif self.is_test_job(job_name): - stage_type = CIStages.TESTS_1 if job_name in CI_CONFIG.test_configs: required_build = CI_CONFIG.test_configs[job_name].required_build assert required_build @@ -591,6 +591,8 @@ class CIConfig: stage_type = CIStages.TESTS_2 else: stage_type = CIStages.TESTS_1 + if job_name not in REQUIRED_CHECKS: + stage_type = CIStages.TESTS_3 assert stage_type, f"BUG [{job_name}]" return stage_type diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 0b51d98b479..0ca25f39976 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -469,7 +469,10 @@ def update_mergeable_check( def trigger_mergeable_check( - commit: Commit, statuses: CommitStatuses, hide_url: bool = False + commit: Commit, + statuses: CommitStatuses, + hide_url: bool = False, + set_if_green: bool = False, ) -> CommitStatus: """calculate and update StatusNames.MERGEABLE""" required_checks = [ @@ -502,6 +505,10 @@ def trigger_mergeable_check( state = FAILURE description = format_description(description) + if not set_if_green and state == SUCCESS: + # do not set green Mergeable Check status + return SUCCESS + if mergeable_status is None or mergeable_status.description != description: return set_mergeable_check(commit, description, state, hide_url) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 450ece62d4b..519fa5fcebb 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -13,7 +13,11 @@ from github.PaginatedList import PaginatedList from github.PullRequestReview import PullRequestReview from github.WorkflowRun import WorkflowRun -from commit_status_helper import get_commit_filtered_statuses +from commit_status_helper import ( + get_commit_filtered_statuses, + get_commit, + trigger_mergeable_check, +) from get_robot_token import get_best_robot_token from github_helper import GitHub, NamedUser, PullRequest, Repository from pr_info import PRInfo @@ -173,6 +177,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="if set, the script won't merge the PR, just check the conditions", ) + parser.add_argument( + "--set-ci-status", + action="store_true", + help="if set, only update/set Mergeable Check status", + ) parser.add_argument( "--check-approved", action="store_true", @@ -226,6 +235,14 @@ def main(): token = args.token or get_best_robot_token() gh = GitHub(token) repo = gh.get_repo(args.repo) + + if args.set_ci_status: + # set mergeable check status and exit + commit = get_commit(gh, args.pr_info.sha) + statuses = get_commit_filtered_statuses(commit) + trigger_mergeable_check(commit, statuses, hide_url=False, set_if_green=True) + return + # An ugly and not nice fix to patch the wrong organization URL, # see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710 # pylint: disable=protected-access From d5eac97d458c0177a6d3d4bb2b603ef1d14feed1 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 18 May 2024 19:13:34 +0200 Subject: [PATCH 31/85] remove update_mergeable_check_from_ci.py --- .github/workflows/pull_request.yml | 1 - tests/ci/ci.py | 40 +----------------------------- tests/ci/commit_status_helper.py | 16 +++++------- 3 files changed, 7 insertions(+), 50 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index e0eda476d19..a9570bc2674 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -174,7 +174,6 @@ jobs: with: filter: tree:0 - name: Check and set merge status - if: ${{ github.event_name != 'merge_group' }} run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 merge_pr.py --set-status diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 08048564383..3aa8f1bb813 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames +from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames from ci_utils import GHActions, is_hex, normalize_string from clickhouse_helper import ( CiLogsCredentials, @@ -34,16 +34,12 @@ from commit_status_helper import ( get_commit, post_commit_status, set_status_comment, - update_mergeable_check, - update_upstream_sync_status, ) from digest_helper import DockerDigester, JobDigester from env_helper import ( CI, GITHUB_JOB_API_URL, - GITHUB_REPOSITORY, GITHUB_RUN_URL, - GITHUB_UPSTREAM_REPOSITORY, REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, @@ -56,7 +52,6 @@ from github_helper import GitHub from pr_info import PRInfo from report import ERROR, SUCCESS, BuildResult, JobReport from s3_helper import S3Helper -from synchronizer_utils import SYNC_BRANCH_PREFIX from version_helper import get_version_from_repo # pylint: disable=too-many-lines @@ -2189,39 +2184,6 @@ def main() -> int: pr_info, dump_to_file=True, ) - if not pr_info.is_merge_queue: - # in the merge queue mergeable status must be set only in FinishCheck (last job in wf) - mergeable_status = update_mergeable_check( - commit, - pr_info, - job_report.check_name or _get_ext_check_name(args.job_name), - ) - - # Process upstream StatusNames.SYNC - if ( - pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") - and mergeable_status - and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY - ): - upstream_pr_number = int( - pr_info.head_ref.split("/pr/", maxsplit=1)[1] - ) - update_upstream_sync_status( - upstream_pr_number, pr_info.number, gh, mergeable_status - ) - prepared_events = prepare_tests_results_for_clickhouse( - pr_info, - [], - job_report.status, - 0, - job_report.start_time, - f"https://github.com/ClickHouse/ClickHouse/pull/{upstream_pr_number}", - StatusNames.SYNC, - ) - prepared_events[0]["test_context_raw"] = args.job_name - ch_helper.insert_events_into( - db="default", table="checks", events=prepared_events - ) print(f"Job report url: [{check_url}]") prepared_events = prepare_tests_results_for_clickhouse( diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 0ca25f39976..2ba6fba8b83 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -447,9 +447,7 @@ def set_mergeable_check( ) -def update_mergeable_check( - commit: Commit, pr_info: PRInfo, check_name: str -) -> Optional[CommitStatus]: +def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> None: "check if the check_name in REQUIRED_CHECKS and then trigger update" not_run = ( pr_info.labels.intersection({Labels.SKIP_MERGEABLE_CHECK, Labels.RELEASE}) @@ -460,12 +458,12 @@ def update_mergeable_check( if not_run: # Let's avoid unnecessary work - return None + return logging.info("Update Mergeable Check by %s", check_name) statuses = get_commit_filtered_statuses(commit) - return trigger_mergeable_check(commit, statuses) + trigger_mergeable_check(commit, statuses) def trigger_mergeable_check( @@ -473,7 +471,7 @@ def trigger_mergeable_check( statuses: CommitStatuses, hide_url: bool = False, set_if_green: bool = False, -) -> CommitStatus: +) -> None: """calculate and update StatusNames.MERGEABLE""" required_checks = [ status for status in statuses if status.context in REQUIRED_CHECKS @@ -507,12 +505,10 @@ def trigger_mergeable_check( if not set_if_green and state == SUCCESS: # do not set green Mergeable Check status - return SUCCESS + return if mergeable_status is None or mergeable_status.description != description: - return set_mergeable_check(commit, description, state, hide_url) - - return mergeable_status + set_mergeable_check(commit, description, state, hide_url) def update_upstream_sync_status( From 4e4d078786a6a7a22b1c6d2190c8a414614cafd2 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 18 May 2024 20:36:04 +0200 Subject: [PATCH 32/85] return sync pr status to upstream from FinishCheck job --- .github/workflows/pull_request.yml | 4 ++-- tests/ci/commit_status_helper.py | 18 ++++++++++-------- tests/ci/finish_check.py | 13 ++++++++++++- 3 files changed, 24 insertions(+), 11 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index a9570bc2674..f2e4b5f328d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -165,7 +165,7 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} CheckReadyForMerge: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: @@ -176,7 +176,7 @@ jobs: - name: Check and set merge status run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 merge_pr.py --set-status + python3 merge_pr.py --set-ci-status ################################# Stage Final ################################# # diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 2ba6fba8b83..ec9746e9af9 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -471,7 +471,7 @@ def trigger_mergeable_check( statuses: CommitStatuses, hide_url: bool = False, set_if_green: bool = False, -) -> None: +) -> StatusType: """calculate and update StatusNames.MERGEABLE""" required_checks = [ status for status in statuses if status.context in REQUIRED_CHECKS @@ -505,17 +505,19 @@ def trigger_mergeable_check( if not set_if_green and state == SUCCESS: # do not set green Mergeable Check status - return + pass + else: + if mergeable_status is None or mergeable_status.description != description: + set_mergeable_check(commit, description, state, hide_url) - if mergeable_status is None or mergeable_status.description != description: - set_mergeable_check(commit, description, state, hide_url) + return state def update_upstream_sync_status( upstream_pr_number: int, sync_pr_number: int, gh: Github, - mergeable_status: CommitStatus, + state: StatusType, ) -> None: upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) upstream_pr = upstream_repo.get_pull(upstream_pr_number) @@ -546,19 +548,19 @@ def update_upstream_sync_status( ) return - sync_status = get_status(mergeable_status.state) + sync_status = get_status(state) logging.info( "Using commit %s to post the %s status `%s`: [%s]", upstream_commit.sha, sync_status, StatusNames.SYNC, - mergeable_status.description, + "", ) post_commit_status( upstream_commit, sync_status, "", # let's won't expose any urls from cloud - mergeable_status.description, + "", StatusNames.SYNC, ) trigger_mergeable_check( diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index a66ebbeadf4..615b26b51f0 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -11,10 +11,13 @@ from commit_status_helper import ( post_commit_status, set_mergeable_check, trigger_mergeable_check, + update_upstream_sync_status, ) from get_robot_token import get_best_robot_token from pr_info import PRInfo from report import PENDING, SUCCESS +from synchronizer_utils import SYNC_BRANCH_PREFIX +from tests.ci.env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY def main(): @@ -40,7 +43,15 @@ def main(): set_mergeable_check(commit, "workflow passed", "success") else: statuses = get_commit_filtered_statuses(commit) - trigger_mergeable_check(commit, statuses) + state = trigger_mergeable_check(commit, statuses, set_if_green=True) + + # Process upstream StatusNames.SYNC + if ( + pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") + and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY + ): + upstream_pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) + update_upstream_sync_status(upstream_pr_number, pr_info.number, gh, state) statuses = [s for s in statuses if s.context == StatusNames.CI] if not statuses: From 714420fc6713d8e1f1a6af29bd37ad932d86059f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 09:00:35 +0200 Subject: [PATCH 33/85] Speed up Set index a little --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 74 ++++++-------------- src/Storages/MergeTree/MergeTreeIndexSet.h | 1 - 2 files changed, 23 insertions(+), 52 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 1bd42518fdd..0b7e2e1f942 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -35,8 +35,7 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( size_t max_rows_) : index_name(index_name_) , max_rows(max_rows_) - , index_sample_block(index_sample_block_) - , block(index_sample_block) + , block(index_sample_block_) { } @@ -47,8 +46,7 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( MutableColumns && mutable_columns_) : index_name(index_name_) , max_rows(max_rows_) - , index_sample_block(index_sample_block_) - , block(index_sample_block.cloneWithColumns(std::move(mutable_columns_))) + , block(index_sample_block_.cloneWithColumns(std::move(mutable_columns_))) { } @@ -67,10 +65,11 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const } size_serialization->serializeBinary(size(), ostr, {}); + size_t num_columns = block.columns(); - for (size_t i = 0; i < index_sample_block.columns(); ++i) + for (size_t i = 0; i < num_columns; ++i) { - const auto & type = index_sample_block.getByPosition(i).type; + const auto & type = block.getByPosition(i).type; ISerialization::SerializeBinaryBulkSettings settings; settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; @@ -92,8 +91,6 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd if (version != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); - block.clear(); - Field field_rows; const auto & size_type = DataTypePtr(std::make_shared()); size_type->getDefaultSerialization()->deserializeBinary(field_rows, istr, {}); @@ -102,24 +99,22 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd if (rows_to_read == 0) return; - for (size_t i = 0; i < index_sample_block.columns(); ++i) + size_t num_columns = block.columns(); + + ISerialization::DeserializeBinaryBulkSettings settings; + settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; + settings.position_independent_encoding = false; + + for (size_t i = 0; i < num_columns; ++i) { - const auto & column = index_sample_block.getByPosition(i); - const auto & type = column.type; - ColumnPtr new_column = type->createColumn(); - - - ISerialization::DeserializeBinaryBulkSettings settings; - settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; - settings.position_independent_encoding = false; + auto & elem = block.getByPosition(i); + elem.column = elem.column->cloneEmpty(); ISerialization::DeserializeBinaryBulkStatePtr state; - auto serialization = type->getDefaultSerialization(); + auto serialization = elem.type->getDefaultSerialization(); serialization->deserializeBinaryBulkStatePrefix(settings, state); - serialization->deserializeBinaryBulkWithMultipleStreams(new_column, rows_to_read, settings, state, nullptr); - - block.insert(ColumnWithTypeAndName(new_column, type, column.name)); + serialization->deserializeBinaryBulkWithMultipleStreams(elem.column, rows_to_read, settings, state, nullptr); } } @@ -284,42 +279,19 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx if (isUseless()) return true; - auto granule = std::dynamic_pointer_cast(idx_granule); - if (!granule) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Set index condition got a granule with the wrong type"); + const MergeTreeIndexGranuleSet & granule = assert_cast(*idx_granule); - if (isUseless() || granule->empty() || (max_rows != 0 && granule->size() > max_rows)) + size_t size = granule.size(); + if (size == 0 || (max_rows != 0 && size > max_rows)) return true; - Block result = granule->block; + Block result = granule.block; actions->execute(result); - const auto & filter_node_name = actions->getActionsDAG().getOutputs().at(0)->result_name; - auto column = result.getByName(filter_node_name).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); + const auto & column = result.getByPosition(result.columns() - 1).column; - if (column->onlyNull()) - return false; - - const auto * col_uint8 = typeid_cast(column.get()); - - const NullMap * null_map = nullptr; - - if (const auto * col_nullable = checkAndGetColumn(&*column)) - { - col_uint8 = typeid_cast(&col_nullable->getNestedColumn()); - null_map = &col_nullable->getNullMapData(); - } - - if (!col_uint8) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "ColumnUInt8 expected as Set index condition result"); - - const auto & condition = col_uint8->getData(); - size_t column_size = column->size(); - - for (size_t i = 0; i < column_size; ++i) - if ((!null_map || (*null_map)[i] == 0) && condition[i] & 1) + for (size_t i = 0; i < size; ++i) + if (column->getBool(i)) return true; return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 7c66ba1a867..3348b5fbe34 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -34,7 +34,6 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule const String index_name; const size_t max_rows; - const Block index_sample_block; Block block; }; From 500475f2b81e74276f6316e710ff7313244928e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 10:45:05 +0200 Subject: [PATCH 34/85] Add a test --- tests/performance/set_index_analysis.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 tests/performance/set_index_analysis.xml diff --git a/tests/performance/set_index_analysis.xml b/tests/performance/set_index_analysis.xml new file mode 100644 index 00000000000..64d0af6690b --- /dev/null +++ b/tests/performance/set_index_analysis.xml @@ -0,0 +1,14 @@ + + + CREATE TABLE test_set (k UInt32, x UInt32, INDEX idx (x) TYPE set(10) GRANULARITY 1) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 111; + + SYSTEM STOP MERGES + INSERT INTO test_set SELECT number, number DIV 100 + rand() % 7 FROM numbers(3000000) SETTINGS max_insert_threads = 4; + + + SELECT count() FROM test_set WHERE x = 1234 SETTINGS max_threads = 8; + + + SYSTEM START MERGES + DROP TABLE IF EXISTS test_set + From 2a9795f4e39e6b8e2ef0aee3d2e97f396416662e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 10:45:19 +0200 Subject: [PATCH 35/85] Minor changes --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index de769c59d33..949807bb88b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1296,8 +1296,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( size_t last_index_mark = 0; PostingsCacheForStore cache_in_store; - - if (dynamic_cast(&*index_helper) != nullptr) + if (dynamic_cast(index_helper.get())) cache_in_store.store = GinIndexStoreFactory::instance().get(index_helper->getFileName(), part->getDataPartStoragePtr()); for (size_t i = 0; i < ranges.size(); ++i) @@ -1315,12 +1314,12 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) { - // vector of indexes of useful ranges + /// An array of indices of useful ranges. auto result = ann_condition->getUsefulRanges(granule); for (auto range : result) { - // range for corresponding index + /// The range for the corresponding index. MarkRange data_range( std::max(ranges[i].begin, index_mark * index_granularity + range), std::min(ranges[i].end, index_mark * index_granularity + range + 1)); @@ -1344,8 +1343,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( continue; MarkRange data_range( - std::max(ranges[i].begin, index_mark * index_granularity), - std::min(ranges[i].end, (index_mark + 1) * index_granularity)); + std::max(ranges[i].begin, index_mark * index_granularity), + std::min(ranges[i].end, (index_mark + 1) * index_granularity)); if (res.empty() || data_range.begin - res.back().end > min_marks_for_seek) res.push_back(data_range); From 2909e6451b6e0118679b2b96b1c3e26e1e226870 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 08:02:06 +0000 Subject: [PATCH 36/85] Move StringUtils.h/cpp back to Common/ --- programs/CMakeLists.txt | 2 +- programs/client/CMakeLists.txt | 1 - programs/format/Format.cpp | 2 +- programs/git-import/git-import.cpp | 2 +- programs/keeper/clickhouse-keeper.cpp | 2 +- .../library-bridge/ExternalDictionaryLibraryUtils.h | 2 +- programs/main.cpp | 2 +- programs/obfuscator/Obfuscator.cpp | 2 +- programs/odbc-bridge/validateODBCConnectionString.cpp | 2 +- programs/server/CMakeLists.txt | 1 - src/Access/User.cpp | 2 +- src/Access/UsersConfigAccessStorage.cpp | 2 +- .../Combinators/AggregateFunctionCombinatorFactory.cpp | 2 +- src/Backups/BackupImpl.cpp | 2 +- src/Bridge/IBridge.cpp | 2 +- src/CMakeLists.txt | 4 ---- src/Client/ClientBase.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Common/CMakeLists.txt | 2 -- src/Common/Config/CMakeLists.txt | 4 ---- src/Common/Config/ConfigProcessor.cpp | 2 +- src/Common/FrequencyHolder.h | 2 +- src/Common/HTTPHeaderFilter.cpp | 2 +- src/Common/ProxyConfigurationResolverProvider.cpp | 2 +- src/Common/ProxyListConfigurationResolver.cpp | 2 +- src/Common/RemoteHostFilter.cpp | 2 +- src/Common/Scheduler/Nodes/DynamicResourceManager.cpp | 2 +- src/Common/SensitiveDataMasker.cpp | 2 +- src/Common/StringSearcher.h | 2 +- src/Common/{StringUtils => }/StringUtils.cpp | 2 +- src/Common/{StringUtils => }/StringUtils.h | 10 +++++----- src/Common/StringUtils/CMakeLists.txt | 8 -------- src/Common/TLDListsHolder.cpp | 2 +- src/Common/UTF8Helpers.cpp | 2 +- src/Common/Volnitsky.h | 2 +- src/Common/ZooKeeper/CMakeLists.txt | 4 ---- src/Common/ZooKeeper/TestKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 2 +- src/Common/ZooKeeper/examples/CMakeLists.txt | 2 +- .../examples/zkutil_test_commands_new_lib.cpp | 2 +- src/Common/escapeForFileName.cpp | 2 +- src/Common/format.h | 2 +- src/Common/formatIPv6.cpp | 2 +- src/Common/formatIPv6.h | 2 +- src/Common/getMappedArea.cpp | 2 +- src/Common/getMultipleKeysFromConfig.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/DataTypes/DataTypeFactory.cpp | 2 +- src/DataTypes/DataTypeMap.cpp | 2 +- src/DataTypes/DataTypeTuple.cpp | 2 +- src/DataTypes/NestedUtils.cpp | 2 +- src/DataTypes/Serializations/SerializationMap.cpp | 2 +- src/Dictionaries/CMakeLists.txt | 1 - src/Dictionaries/DictionaryStructure.cpp | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Disks/IVolume.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/Disks/VolumeJBOD.cpp | 2 +- src/Formats/CapnProtoSchema.cpp | 2 +- src/Formats/StructureToCapnProtoSchema.cpp | 2 +- src/Formats/StructureToProtobufSchema.cpp | 2 +- src/Functions/ExtractString.h | 2 +- src/Functions/FunctionTokens.h | 2 +- src/Functions/FunctionsProgrammingClassification.cpp | 2 +- src/Functions/FunctionsTonalityClassification.cpp | 2 +- .../JSONPath/Parsers/ParserJSONPathMemberAccess.cpp | 2 +- src/Functions/URL/domain.h | 2 +- src/Functions/URL/netloc.cpp | 2 +- src/Functions/URL/port.cpp | 2 +- src/Functions/URL/protocol.h | 2 +- .../ExternalUserDefinedExecutableFunctionsLoader.cpp | 2 +- .../UserDefined/UserDefinedSQLObjectsDiskStorage.cpp | 2 +- src/Functions/alphaTokens.cpp | 2 +- src/Functions/arrayStringConcat.cpp | 2 +- src/Functions/decodeHTMLComponent.cpp | 2 +- src/Functions/decodeXMLComponent.cpp | 2 +- src/Functions/extractAll.cpp | 2 +- src/Functions/extractTextFromHTML.cpp | 2 +- src/Functions/initcap.cpp | 2 +- src/Functions/soundex.cpp | 2 +- src/Functions/splitByChar.cpp | 2 +- src/Functions/splitByNonAlpha.cpp | 2 +- src/Functions/splitByRegexp.cpp | 2 +- src/Functions/splitByString.cpp | 2 +- src/Functions/splitByWhitespace.cpp | 2 +- src/Functions/translate.cpp | 2 +- src/IO/HTTPChunkedReadBuffer.cpp | 2 +- src/IO/ReadHelpers.cpp | 2 +- src/IO/ReadHelpers.h | 2 +- src/IO/S3Common.cpp | 2 +- src/IO/WriteHelpers.h | 2 +- src/IO/parseDateTimeBestEffort.cpp | 2 +- src/IO/readFloatText.h | 2 +- .../Access/InterpreterShowAccessEntitiesQuery.cpp | 2 +- .../Access/InterpreterShowCreateAccessEntityQuery.cpp | 2 +- src/Interpreters/Cluster.cpp | 2 +- src/Interpreters/ClusterDiscovery.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExternalLoader.cpp | 2 +- src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 2 +- src/Interpreters/ITokenExtractor.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterserverCredentials.cpp | 2 +- src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 2 +- src/Interpreters/QueryNormalizer.cpp | 2 +- src/Interpreters/TableJoin.cpp | 2 +- src/Interpreters/TranslateQualifiedNamesVisitor.cpp | 2 +- src/Interpreters/misc.h | 2 +- src/Parsers/CMakeLists.txt | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ExpressionListParsers.cpp | 2 +- .../Kusto/KustoFunctions/KQLAggregationFunctions.cpp | 2 +- src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp | 2 +- src/Parsers/Kusto/parseKQLQuery.cpp | 2 +- src/Parsers/Lexer.cpp | 2 +- src/Parsers/ParserDataType.cpp | 2 +- src/Parsers/formatSettingName.cpp | 2 +- src/Parsers/obfuscateQueries.cpp | 2 +- src/Parsers/parseQuery.cpp | 2 +- src/Parsers/queryNormalization.cpp | 2 +- .../Merges/Algorithms/SummingSortedAlgorithm.cpp | 2 +- src/Server/HTTPHandler.cpp | 2 +- src/Server/HTTPHandlerFactory.h | 2 +- src/Server/HTTPHandlerRequestFilter.h | 2 +- src/Storages/CompressionCodecSelector.h | 2 +- .../DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/IStorage.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/ReplicatedMergeTreePartHeader.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/RocksDB/StorageSystemRocksDB.cpp | 2 +- src/Storages/StorageFactory.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/System/CMakeLists.txt | 1 - src/Storages/System/StorageSystemDashboards.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/examples/CMakeLists.txt | 2 +- 144 files changed, 139 insertions(+), 165 deletions(-) rename src/Common/{StringUtils => }/StringUtils.cpp (95%) rename src/Common/{StringUtils => }/StringUtils.h (96%) delete mode 100644 src/Common/StringUtils/CMakeLists.txt diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 0d91de2dad8..4640882f2be 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -162,7 +162,7 @@ if (ARCH_AMD64 AND OS_LINUX AND NOT OS_ANDROID) set (HARMFUL_LIB harmful) endif () -target_link_libraries (clickhouse PRIVATE clickhouse_common_io string_utils ${HARMFUL_LIB}) +target_link_libraries (clickhouse PRIVATE clickhouse_common_io ${HARMFUL_LIB}) target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) if (ENABLE_CLICKHOUSE_KEEPER) diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index e160355ef7b..f8ef8ccaf65 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -10,7 +10,6 @@ set (CLICKHOUSE_CLIENT_LINK clickhouse_common_io clickhouse_functions clickhouse_parsers - string_utils ) if (TARGET ch_rust::skim) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index d4b975ce1e8..1b91e7ceaf3 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index eaf85df67b1..5430c4b0a42 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/keeper/clickhouse-keeper.cpp b/programs/keeper/clickhouse-keeper.cpp index be2686d936b..f2f91930ac0 100644 --- a/programs/keeper/clickhouse-keeper.cpp +++ b/programs/keeper/clickhouse-keeper.cpp @@ -1,4 +1,4 @@ -#include +#include #include "config_tools.h" diff --git a/programs/library-bridge/ExternalDictionaryLibraryUtils.h b/programs/library-bridge/ExternalDictionaryLibraryUtils.h index e6bf8f2a4c3..2eb44022742 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryUtils.h +++ b/programs/library-bridge/ExternalDictionaryLibraryUtils.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/programs/main.cpp b/programs/main.cpp index 4bb73399719..bc8476e4ce4 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -15,7 +15,7 @@ #include "config_tools.h" -#include +#include #include #include diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 8035f053b41..688ae1a1143 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/odbc-bridge/validateODBCConnectionString.cpp b/programs/odbc-bridge/validateODBCConnectionString.cpp index 6c6e11162b4..72c3c9bddca 100644 --- a/programs/odbc-bridge/validateODBCConnectionString.cpp +++ b/programs/odbc-bridge/validateODBCConnectionString.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include "validateODBCConnectionString.h" diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 81440b03690..76d201cc924 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -13,7 +13,6 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_parsers clickhouse_storages_system clickhouse_table_functions - string_utils ${LINK_RESOURCE_LIB} diff --git a/src/Access/User.cpp b/src/Access/User.cpp index ef5cf722113..6a296706baf 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 908ff780c62..1f9a977bab6 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.cpp index a42e4177ac5..428f7168826 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.cpp @@ -1,6 +1,6 @@ #include "AggregateFunctionCombinatorFactory.h" -#include +#include namespace DB { diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 8c0989b8202..8f32c918c61 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index 6da2b7c06da..c25d7bd2fed 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index da17bc1f41f..4e8946facda 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -307,7 +307,6 @@ endif() target_link_libraries (clickhouse_common_io PRIVATE - string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} PUBLIC @@ -320,7 +319,6 @@ target_link_libraries (clickhouse_common_io target_link_libraries (clickhouse_compression PUBLIC - string_utils pcg_random clickhouse_parsers PRIVATE @@ -410,7 +408,6 @@ dbms_target_link_libraries ( clickhouse_parsers ch_contrib::lz4 Poco::JSON - string_utils PUBLIC boost::system clickhouse_common_io @@ -645,7 +642,6 @@ if (ENABLE_TESTS) dbms clickhouse_common_config clickhouse_common_zookeeper - string_utils hilite_comparator) if (TARGET ch_contrib::simdjson) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 67aba2256e8..085d95370ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6e626c22527..19cd8cc4ee5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index b83c8431f0a..d4802c28f53 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -1,5 +1,3 @@ -add_subdirectory(StringUtils) - if (ENABLE_BENCHMARKS) add_subdirectory(benchmarks) endif() diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 009e2456322..09095ef5acc 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -13,8 +13,6 @@ target_link_libraries(clickhouse_common_config clickhouse_common_zookeeper common Poco::XML - PRIVATE - string_utils ) add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) @@ -23,8 +21,6 @@ target_link_libraries(clickhouse_common_config_no_zookeeper_log clickhouse_common_zookeeper_no_log common Poco::XML - PRIVATE - string_utils ) if (TARGET ch_contrib::yaml_cpp) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 7930ef20153..c9832e8efd5 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/FrequencyHolder.h b/src/Common/FrequencyHolder.h index 64207dc5423..d6c32c225bf 100644 --- a/src/Common/FrequencyHolder.h +++ b/src/Common/FrequencyHolder.h @@ -12,7 +12,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Common/HTTPHeaderFilter.cpp b/src/Common/HTTPHeaderFilter.cpp index 9ad8dd6fccf..fd02fe1ecef 100644 --- a/src/Common/HTTPHeaderFilter.cpp +++ b/src/Common/HTTPHeaderFilter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index d15b4d98615..1a6dc1090ee 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index c9b8923929a..c527c89ea6b 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/src/Common/RemoteHostFilter.cpp b/src/Common/RemoteHostFilter.cpp index 815be8902e6..fe7bf878596 100644 --- a/src/Common/RemoteHostFilter.cpp +++ b/src/Common/RemoteHostFilter.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index b568b9245ba..01aa7df48d3 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 8c29b899841..a9f61a1c786 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #ifndef NDEBUG diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index b3065354f65..d7e706fcd80 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Common/StringUtils/StringUtils.cpp b/src/Common/StringUtils.cpp similarity index 95% rename from src/Common/StringUtils/StringUtils.cpp rename to src/Common/StringUtils.cpp index 8a0b25dbdad..f61a39851e2 100644 --- a/src/Common/StringUtils/StringUtils.cpp +++ b/src/Common/StringUtils.cpp @@ -1,7 +1,7 @@ #include "StringUtils.h" -namespace detail +namespace impl { bool startsWith(const std::string & s, const char * prefix, size_t prefix_size) diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils.h similarity index 96% rename from src/Common/StringUtils/StringUtils.h rename to src/Common/StringUtils.h index 4958ecc9476..051e4338714 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils.h @@ -8,7 +8,7 @@ #include -namespace detail +namespace impl { bool startsWith(const std::string & s, const char * prefix, size_t prefix_size); bool endsWith(const std::string & s, const char * suffix, size_t suffix_size); @@ -17,12 +17,12 @@ namespace detail inline bool startsWith(const std::string & s, const std::string & prefix) { - return detail::startsWith(s, prefix.data(), prefix.size()); + return impl::startsWith(s, prefix.data(), prefix.size()); } inline bool endsWith(const std::string & s, const std::string & suffix) { - return detail::endsWith(s, suffix.data(), suffix.size()); + return impl::endsWith(s, suffix.data(), suffix.size()); } @@ -30,12 +30,12 @@ inline bool endsWith(const std::string & s, const std::string & suffix) /// string that is known at compile time. inline bool startsWith(const std::string & s, const char * prefix) { - return detail::startsWith(s, prefix, strlen(prefix)); + return impl::startsWith(s, prefix, strlen(prefix)); } inline bool endsWith(const std::string & s, const char * suffix) { - return detail::endsWith(s, suffix, strlen(suffix)); + return impl::endsWith(s, suffix, strlen(suffix)); } /// Given an integer, return the adequate suffix for diff --git a/src/Common/StringUtils/CMakeLists.txt b/src/Common/StringUtils/CMakeLists.txt deleted file mode 100644 index 57c196d335c..00000000000 --- a/src/Common/StringUtils/CMakeLists.txt +++ /dev/null @@ -1,8 +0,0 @@ -# These files are located in separate library, because they are used by separate products -# in places when no dependency on whole "dbms" library is possible. - -include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") - -add_headers_and_sources(clickhouse_common_stringutils .) - -add_library(string_utils ${clickhouse_common_stringutils_headers} ${clickhouse_common_stringutils_sources}) diff --git a/src/Common/TLDListsHolder.cpp b/src/Common/TLDListsHolder.cpp index c3991b86983..413d221090e 100644 --- a/src/Common/TLDListsHolder.cpp +++ b/src/Common/TLDListsHolder.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index b8f5c000e75..3c3cf61bbfc 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index 6513bdb8bc3..3a148983790 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index aa06375bd6a..8b6c420e565 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -12,8 +12,6 @@ target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io clickhouse_compression common - PRIVATE - string_utils ) # for examples -- no logging (to avoid extra dependencies) @@ -23,8 +21,6 @@ target_link_libraries (clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_compression common - PRIVATE - string_utils ) if (ENABLE_EXAMPLES) add_subdirectory(examples) diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index d02ad4523ad..51ad2e7c830 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -1,7 +1,7 @@ #include "Common/ZooKeeper/IKeeper.h" #include #include -#include +#include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index c62c5d0c143..be490d0bfc1 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -19,7 +19,7 @@ #include #include "Common/ZooKeeper/IKeeper.h" #include -#include +#include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 40bd9d79a03..a581b6a7f38 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index a99fbe55dd8..16300115362 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -2,7 +2,7 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log clickhouse_compression string_utils) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log clickhouse_compression) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp index 414006d48a4..25d66b94b46 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/escapeForFileName.cpp b/src/Common/escapeForFileName.cpp index a1f9bff28d0..2fe23245f49 100644 --- a/src/Common/escapeForFileName.cpp +++ b/src/Common/escapeForFileName.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Common/format.h b/src/Common/format.h index 27018f64064..3dbb88b4089 100644 --- a/src/Common/format.h +++ b/src/Common/format.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Common/formatIPv6.cpp b/src/Common/formatIPv6.cpp index 86e33beb7c3..341b3715d30 100644 --- a/src/Common/formatIPv6.cpp +++ b/src/Common/formatIPv6.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index 3451eda6b3c..bb83e0381ef 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include constexpr size_t IPV4_BINARY_LENGTH = 4; constexpr size_t IPV6_BINARY_LENGTH = 16; diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp index 4f40c604c6a..79191d68fb9 100644 --- a/src/Common/getMappedArea.cpp +++ b/src/Common/getMappedArea.cpp @@ -3,7 +3,7 @@ #if defined(OS_LINUX) -#include +#include #include #include #include diff --git a/src/Common/getMultipleKeysFromConfig.cpp b/src/Common/getMultipleKeysFromConfig.cpp index 7cf49fcc34d..6d6589a45a3 100644 --- a/src/Common/getMultipleKeysFromConfig.cpp +++ b/src/Common/getMultipleKeysFromConfig.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 25254e10441..28902bc8591 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include "Coordination/KeeperFeatureFlags.h" #include diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index df5c2e9e0c8..9bcd0608bf7 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 844384f3c95..427af090b91 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index d9f70e1659d..4d7ab63f966 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 5bbd79160d4..ebee096613d 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 6a56f885503..650559d21d9 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include "Columns/IColumn.h" #include diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 7b6f87baf2e..49bc89687f1 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index 569acd9231a..783835356e6 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -39,7 +39,6 @@ target_link_libraries(clickhouse_dictionaries Poco::Data Poco::MongoDB Poco::Redis - string_utils ) target_link_libraries(clickhouse_dictionaries PUBLIC ch_contrib::abseil_swiss_tables) diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 0b6bdea60a3..c2f2f4a8532 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 16a4ecaee75..fde46fb27f0 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,6 +1,6 @@ #include "FileDictionarySource.h" #include -#include +#include #include #include #include diff --git a/src/Disks/IVolume.cpp b/src/Disks/IVolume.cpp index d763c55c4aa..e6be0f36193 100644 --- a/src/Disks/IVolume.cpp +++ b/src/Disks/IVolume.cpp @@ -1,6 +1,6 @@ #include "IVolume.h" -#include +#include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 043e5b8ef8c..adbdd9d13aa 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -20,7 +20,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index c3114eb0b6f..35913613326 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -4,7 +4,7 @@ #if USE_AWS_S3 -#include +#include #include #include #include diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index a0c71583a22..d0e9d32ff5e 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -1,6 +1,6 @@ #include "VolumeJBOD.h" -#include +#include #include #include #include diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp index 559047a6f8d..6076dae4157 100644 --- a/src/Formats/CapnProtoSchema.cpp +++ b/src/Formats/CapnProtoSchema.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Formats/StructureToCapnProtoSchema.cpp b/src/Formats/StructureToCapnProtoSchema.cpp index 99298fadee1..cd45b19d3c0 100644 --- a/src/Formats/StructureToCapnProtoSchema.cpp +++ b/src/Formats/StructureToCapnProtoSchema.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Formats/StructureToProtobufSchema.cpp b/src/Formats/StructureToProtobufSchema.cpp index 178c0ae3cc2..9fd02969adb 100644 --- a/src/Formats/StructureToProtobufSchema.cpp +++ b/src/Formats/StructureToProtobufSchema.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Functions/ExtractString.h b/src/Functions/ExtractString.h index aa0e1b04835..6beb8be830a 100644 --- a/src/Functions/ExtractString.h +++ b/src/Functions/ExtractString.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h index c80152bc71d..d6cf6a24983 100644 --- a/src/Functions/FunctionTokens.h +++ b/src/Functions/FunctionTokens.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/FunctionsProgrammingClassification.cpp b/src/Functions/FunctionsProgrammingClassification.cpp index a93e1d9a87d..c01e47ad0d7 100644 --- a/src/Functions/FunctionsProgrammingClassification.cpp +++ b/src/Functions/FunctionsProgrammingClassification.cpp @@ -2,7 +2,7 @@ #if USE_NLP -#include +#include #include #include diff --git a/src/Functions/FunctionsTonalityClassification.cpp b/src/Functions/FunctionsTonalityClassification.cpp index 3de38d99c88..a9321819a26 100644 --- a/src/Functions/FunctionsTonalityClassification.cpp +++ b/src/Functions/FunctionsTonalityClassification.cpp @@ -2,7 +2,7 @@ #if USE_NLP -#include +#include #include #include diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp index 6d5e37623e9..fad822379d4 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Functions/URL/domain.h b/src/Functions/URL/domain.h index 87f5aeffda7..936fb9d5f00 100644 --- a/src/Functions/URL/domain.h +++ b/src/Functions/URL/domain.h @@ -3,7 +3,7 @@ #include "protocol.h" #include #include -#include +#include namespace DB { diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index abfa7ec26fd..d1ca4fa1614 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index 942f6b702fd..c8f50f10a56 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Functions/URL/protocol.h b/src/Functions/URL/protocol.h index c1d83192835..5e90f538ff1 100644 --- a/src/Functions/URL/protocol.h +++ b/src/Functions/URL/protocol.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index a4f17aa1201..2c031158c48 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -1,7 +1,7 @@ #include "ExternalUserDefinedExecutableFunctionsLoader.h" #include -#include +#include #include diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp index d874612ad04..b406cc8d317 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp @@ -3,7 +3,7 @@ #include "Functions/UserDefined/UserDefinedSQLFunctionFactory.h" #include "Functions/UserDefined/UserDefinedSQLObjectType.h" -#include +#include #include #include #include diff --git a/src/Functions/alphaTokens.cpp b/src/Functions/alphaTokens.cpp index 35f434e7498..f4d77f1d654 100644 --- a/src/Functions/alphaTokens.cpp +++ b/src/Functions/alphaTokens.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Functions/arrayStringConcat.cpp b/src/Functions/arrayStringConcat.cpp index b787feeeca1..421408c01f2 100644 --- a/src/Functions/arrayStringConcat.cpp +++ b/src/Functions/arrayStringConcat.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Functions/decodeHTMLComponent.cpp b/src/Functions/decodeHTMLComponent.cpp index 4db3c43f946..00a601b77a6 100644 --- a/src/Functions/decodeHTMLComponent.cpp +++ b/src/Functions/decodeHTMLComponent.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index a25e67e0e37..cbbe46fcb8c 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/extractAll.cpp b/src/Functions/extractAll.cpp index f0c18bf79b9..5801a7b8f4f 100644 --- a/src/Functions/extractAll.cpp +++ b/src/Functions/extractAll.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index 4eefeaa9f86..d9aa004b279 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include /** A function to extract text from HTML or XHTML. diff --git a/src/Functions/initcap.cpp b/src/Functions/initcap.cpp index 5460ee06792..6b2958227bc 100644 --- a/src/Functions/initcap.cpp +++ b/src/Functions/initcap.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Functions/soundex.cpp b/src/Functions/soundex.cpp index 0cddfc90f7c..77ddb14a6ec 100644 --- a/src/Functions/soundex.cpp +++ b/src/Functions/soundex.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Functions/splitByChar.cpp b/src/Functions/splitByChar.cpp index d3d5dc9fe4a..52db5623b89 100644 --- a/src/Functions/splitByChar.cpp +++ b/src/Functions/splitByChar.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/src/Functions/splitByNonAlpha.cpp b/src/Functions/splitByNonAlpha.cpp index 4486a33aa88..17ff6cfb0a8 100644 --- a/src/Functions/splitByNonAlpha.cpp +++ b/src/Functions/splitByNonAlpha.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/splitByRegexp.cpp b/src/Functions/splitByRegexp.cpp index 430089f14ee..32afb813a04 100644 --- a/src/Functions/splitByRegexp.cpp +++ b/src/Functions/splitByRegexp.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include diff --git a/src/Functions/splitByString.cpp b/src/Functions/splitByString.cpp index 5c97f9841e7..e9b70a58eab 100644 --- a/src/Functions/splitByString.cpp +++ b/src/Functions/splitByString.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/src/Functions/splitByWhitespace.cpp b/src/Functions/splitByWhitespace.cpp index cf21a218b15..5bf27f64c17 100644 --- a/src/Functions/splitByWhitespace.cpp +++ b/src/Functions/splitByWhitespace.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index c7173909029..2df08a5664e 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/IO/HTTPChunkedReadBuffer.cpp b/src/IO/HTTPChunkedReadBuffer.cpp index 41788fa8ce7..b5ac6a9b728 100644 --- a/src/IO/HTTPChunkedReadBuffer.cpp +++ b/src/IO/HTTPChunkedReadBuffer.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 8c83eac5cff..b428b1c7d8a 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 5cf7d3e5b66..63bfae513e7 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -30,7 +30,7 @@ #include #include -#include +#include #include #include diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 56e3e0df21b..4583b2bb0ac 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include "config.h" diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index a30e2feb439..d4b2d8ea0dc 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -26,7 +26,7 @@ #include #include -#include +#include #include #include diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 70401fdf72d..e046e837689 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index d1652784cc2..3a21d7201a9 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wunneeded-internal-declaration" diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index 76979ed86c8..71fc1047cfa 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 1147d74c146..96d8e55a74c 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index ec6283df649..59c98491c14 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index d432488964d..6f9c375c2f5 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 395218f834f..d80d5cd5b93 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -47,7 +47,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index a636e59fa1a..96405f35f3f 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index a15f918f457..e404797501d 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/ITokenExtractor.cpp b/src/Interpreters/ITokenExtractor.cpp index 9c4027dfa0a..1c5d0d4b6d4 100644 --- a/src/Interpreters/ITokenExtractor.cpp +++ b/src/Interpreters/ITokenExtractor.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 519cbde588f..4fdd804452d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -6,7 +6,7 @@ #include #include "Common/Exception.h" -#include +#include #include #include #include diff --git a/src/Interpreters/InterserverCredentials.cpp b/src/Interpreters/InterserverCredentials.cpp index c344732a262..1327a2ef388 100644 --- a/src/Interpreters/InterserverCredentials.cpp +++ b/src/Interpreters/InterserverCredentials.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 5cda4c982b4..6a3a181ed26 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index f47635a3c3f..a8639906aad 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 1ee8ca14b2f..6191eb73fd4 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 03df7283992..c21c4d34fa8 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index c009808de3f..b77fc5aee1e 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index 3bc1b3a981f..d5653da7b3a 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -7,7 +7,7 @@ add_headers_and_sources(clickhouse_parsers ./Kusto) add_headers_and_sources(clickhouse_parsers ./PRQL) add_headers_and_sources(clickhouse_parsers ./Kusto/KustoFunctions) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) -target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils) +target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access) if (TARGET ch_rust::prql) target_link_libraries(clickhouse_parsers PRIVATE ch_rust::prql) endif () diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 59b586d46a0..416f696323c 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include "Parsers/CommonParsers.h" diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index ee9e199b9b8..7cdfaf988a3 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include diff --git a/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp index 16436d38d32..0eb83b8b5ac 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp +++ b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp index c4f84d576cb..19625f6624d 100644 --- a/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp +++ b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Parsers/Kusto/parseKQLQuery.cpp b/src/Parsers/Kusto/parseKQLQuery.cpp index 34a009873f8..34076168480 100644 --- a/src/Parsers/Kusto/parseKQLQuery.cpp +++ b/src/Parsers/Kusto/parseKQLQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 9ac6e623803..34855a7ce20 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index fdd712f2e68..05c9a2cd306 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Parsers/formatSettingName.cpp b/src/Parsers/formatSettingName.cpp index efbfffddd7b..59973379167 100644 --- a/src/Parsers/formatSettingName.cpp +++ b/src/Parsers/formatSettingName.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 2ed551851e8..074b6797517 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 66bd76687aa..41c51267496 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/queryNormalization.cpp b/src/Parsers/queryNormalization.cpp index 4a9dd8ceb98..4890ad6952d 100644 --- a/src/Parsers/queryNormalization.cpp +++ b/src/Parsers/queryNormalization.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 79b5dae2d6e..df27520856e 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ce80d0c22c6..a677c537622 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ac18c36e6c9..b4c32366463 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h index 15e64cf7f48..de1920bd535 100644 --- a/src/Server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/CompressionCodecSelector.h b/src/Storages/CompressionCodecSelector.h index ad6e943e821..e03d06bacdb 100644 --- a/src/Storages/CompressionCodecSelector.h +++ b/src/Storages/CompressionCodecSelector.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 14866c25365..d471c67553d 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index b532abc9074..920155bf689 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 463ca07ec57..483e949a26f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -34,7 +34,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 167160db317..527dac01b71 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp index 24d907dbad6..9aadc3c3ca7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d6c36d12bf5..9a368bd44f5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index 4406a7c3fd4..5105b190fd9 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 307a0aa001a..9d12a1569d8 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 1ac739f03fd..306ae782d24 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 25c48de94e1..54b2d5ef6fb 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index df5bbdf9f78..378b81c6d18 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 54218351cf1..205a90423bf 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index c3a2e726365..899c3d5cf40 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -47,7 +47,6 @@ add_library(clickhouse_storages_system ${storages_system_sources}) target_link_libraries(clickhouse_storages_system PRIVATE dbms common - string_utils clickhouse_common_zookeeper clickhouse_parsers Poco::JSON diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp index 23d8fcfc481..9682fbc74a1 100644 --- a/src/Storages/System/StorageSystemDashboards.cpp +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index d428d6bd6d0..43398517072 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index cddfc9404d4..b4786b7313b 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -5,4 +5,4 @@ clickhouse_add_executable (merge_selector2 merge_selector2.cpp) target_link_libraries (merge_selector2 PRIVATE dbms) clickhouse_add_executable (get_current_inserts_in_replicated get_current_inserts_in_replicated.cpp) -target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper string_utils) +target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) From 17ce44907677dbfb6aa2ebc27593ffe02c239149 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 09:38:17 +0000 Subject: [PATCH 37/85] Move isAllASCII from UTF8Helpers to StringUtils --- src/Common/StringUtils.cpp | 72 ++++++++++++++++++++++++++++- src/Common/StringUtils.h | 5 ++ src/Common/UTF8Helpers.cpp | 74 +----------------------------- src/Common/UTF8Helpers.h | 4 -- src/Functions/LowerUpperUTF8Impl.h | 3 +- src/Functions/padString.cpp | 5 +- src/Functions/reverseUTF8.cpp | 3 +- src/Functions/substring.cpp | 5 +- src/Functions/substringIndex.cpp | 13 +++--- 9 files changed, 94 insertions(+), 90 deletions(-) diff --git a/src/Common/StringUtils.cpp b/src/Common/StringUtils.cpp index f61a39851e2..18577e64c01 100644 --- a/src/Common/StringUtils.cpp +++ b/src/Common/StringUtils.cpp @@ -1,4 +1,10 @@ -#include "StringUtils.h" +#include + +#include + +#if USE_MULTITARGET_CODE +#include +#endif namespace impl @@ -15,3 +21,67 @@ bool endsWith(const std::string & s, const char * suffix, size_t suffix_size) } } + +DECLARE_DEFAULT_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + UInt8 mask = 0; + for (size_t i = 0; i < size; ++i) + mask |= data[i]; + + return !(mask & 0x80); +}) + +DECLARE_SSE42_SPECIFIC_CODE( +/// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h +bool isAllASCII(const UInt8 * data, size_t size) +{ + __m128i masks = _mm_setzero_si128(); + + size_t i = 0; + for (; i + 16 <= size; i += 16) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(data + i)); + masks = _mm_or_si128(masks, bytes); + } + int mask = _mm_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +DECLARE_AVX2_SPECIFIC_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + __m256i masks = _mm256_setzero_si256(); + + size_t i = 0; + for (; i + 32 <= size; i += 32) + { + __m256i bytes = _mm256_loadu_si256(reinterpret_cast(data + i)); + masks = _mm256_or_si256(masks, bytes); + } + int mask = _mm256_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +bool isAllASCII(const UInt8 * data, size_t size) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(DB::TargetArch::AVX2)) + return TargetSpecific::AVX2::isAllASCII(data, size); + if (isArchSupported(DB::TargetArch::SSE42)) + return TargetSpecific::SSE42::isAllASCII(data, size); +#endif + return TargetSpecific::Default::isAllASCII(data, size); +} diff --git a/src/Common/StringUtils.h b/src/Common/StringUtils.h index 051e4338714..fe5fc3c058f 100644 --- a/src/Common/StringUtils.h +++ b/src/Common/StringUtils.h @@ -7,6 +7,8 @@ #include #include +#include + namespace impl { @@ -315,6 +317,9 @@ inline void trim(std::string & str, char c = ' ') trimLeft(str, c); } +/// If all characters in the string are ASCII, return true +bool isAllASCII(const UInt8 * data, size_t size); + constexpr bool containsGlobs(const std::string & str) { return str.find_first_of("*?{") != std::string::npos; diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 3c3cf61bbfc..8c8c8e8327b 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,14 +1,9 @@ -#include -#include #include +#include #include #include -#if USE_MULTITARGET_CODE -#include -#endif - namespace DB { namespace UTF8 @@ -208,7 +203,6 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l } - size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept { return computeWidthImpl(data, size, prefix, 0); @@ -219,71 +213,5 @@ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, s return computeWidthImpl(data, size, prefix, limit); } - -DECLARE_DEFAULT_CODE( -bool isAllASCII(const UInt8 * data, size_t size) -{ - UInt8 mask = 0; - for (size_t i = 0; i < size; ++i) - mask |= data[i]; - - return !(mask & 0x80); -}) - -DECLARE_SSE42_SPECIFIC_CODE( -/// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h -bool isAllASCII(const UInt8 * data, size_t size) -{ - __m128i masks = _mm_setzero_si128(); - - size_t i = 0; - for (; i + 16 <= size; i += 16) - { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(data + i)); - masks = _mm_or_si128(masks, bytes); - } - int mask = _mm_movemask_epi8(masks); - - UInt8 tail_mask = 0; - for (; i < size; i++) - tail_mask |= data[i]; - - mask |= (tail_mask & 0x80); - return !mask; -}) - -DECLARE_AVX2_SPECIFIC_CODE( -bool isAllASCII(const UInt8 * data, size_t size) -{ - __m256i masks = _mm256_setzero_si256(); - - size_t i = 0; - for (; i + 32 <= size; i += 32) - { - __m256i bytes = _mm256_loadu_si256(reinterpret_cast(data + i)); - masks = _mm256_or_si256(masks, bytes); - } - int mask = _mm256_movemask_epi8(masks); - - UInt8 tail_mask = 0; - for (; i < size; i++) - tail_mask |= data[i]; - - mask |= (tail_mask & 0x80); - return !mask; -}) - -bool isAllASCII(const UInt8* data, size_t size) -{ -#if USE_MULTITARGET_CODE - if (isArchSupported(TargetArch::AVX2)) - return TargetSpecific::AVX2::isAllASCII(data, size); - if (isArchSupported(TargetArch::SSE42)) - return TargetSpecific::SSE42::isAllASCII(data, size); -#endif - return TargetSpecific::Default::isAllASCII(data, size); -} - - } } diff --git a/src/Common/UTF8Helpers.h b/src/Common/UTF8Helpers.h index 933b62c7b63..b09d92bd731 100644 --- a/src/Common/UTF8Helpers.h +++ b/src/Common/UTF8Helpers.h @@ -136,10 +136,6 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept */ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept; - -/// If all the characters in the string are ASCII, return true. -bool isAllASCII(const UInt8* data, size_t size); - } } diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index bb794a0f8ed..eebba7b9d5f 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #ifdef __SSE2__ @@ -94,7 +95,7 @@ struct LowerUpperUTF8Impl if (data.empty()) return; - bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + bool all_ascii = isAllASCII(data.data(), data.size()); if (all_ascii) { LowerUpperImpl::vector(data, offsets, res_data, res_offsets); diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index 0922e0ddb8a..8670c837e21 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -237,8 +238,8 @@ namespace void executeForSource(SourceStrings && strings, const ColumnPtr & column_length, const String & pad_string, StringSink & res_sink) const { const auto & chars = strings.getElements(); - bool all_ascii = UTF8::isAllASCII(reinterpret_cast(pad_string.data()), pad_string.size()) - && UTF8::isAllASCII(chars.data(), chars.size()); + bool all_ascii = isAllASCII(reinterpret_cast(pad_string.data()), pad_string.size()) + && isAllASCII(chars.data(), chars.size()); bool is_actually_utf8 = is_utf8 && !all_ascii; if (!is_actually_utf8) diff --git a/src/Functions/reverseUTF8.cpp b/src/Functions/reverseUTF8.cpp index 4ea861919a1..1aee349fa8d 100644 --- a/src/Functions/reverseUTF8.cpp +++ b/src/Functions/reverseUTF8.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include "reverse.h" @@ -27,7 +28,7 @@ struct ReverseUTF8Impl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + bool all_ascii = isAllASCII(data.data(), data.size()); if (all_ascii) { ReverseImpl::vector(data, offsets, res_data, res_offsets); diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index 122f83d758b..f1dea7db018 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -149,7 +150,7 @@ public: { if (const ColumnString * col = checkAndGetColumn(column_string.get())) { - bool all_ascii = UTF8::isAllASCII(col->getChars().data(), col->getChars().size()); + bool all_ascii = isAllASCII(col->getChars().data(), col->getChars().size()); if (all_ascii) return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count); else @@ -159,7 +160,7 @@ public: if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) { StringRef str_ref = col_const->getDataAt(0); - bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str_ref.data), str_ref.size); + bool all_ascii = isAllASCII(reinterpret_cast(str_ref.data), str_ref.size); if (all_ascii) return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); else diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 74474cb4b23..15a321bd5b0 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -129,8 +130,8 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); - bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) - && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); + bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); @@ -162,8 +163,8 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); - bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) - && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); + bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); @@ -194,8 +195,8 @@ namespace res_data.reserve(str.size() * rows / 2); res_offsets.reserve(rows); - bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str.data()), str.size()) - && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); + bool all_ascii = isAllASCII(reinterpret_cast(str.data()), str.size()) + && isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); From 332ec7c51fe260d43bcd9b9480daaa2e95179dcb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 14:28:04 +0300 Subject: [PATCH 38/85] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 0b7e2e1f942..e9dc638341a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -35,7 +35,7 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( size_t max_rows_) : index_name(index_name_) , max_rows(max_rows_) - , block(index_sample_block_) + , block(index_sample_block_.cloneEmpty()) { } From 31f0b2f741e8a8c7b06e2271cfd5838a8d16fb32 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 14:52:51 +0300 Subject: [PATCH 39/85] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index e9dc638341a..797455816f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -291,7 +291,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx const auto & column = result.getByPosition(result.columns() - 1).column; for (size_t i = 0; i < size; ++i) - if (column->getBool(i)) + if (column->getUInt(i) & 1) return true; return false; From e18fa68f3d72a0dbed4257c4922a6c534fdb677e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 15:00:14 +0300 Subject: [PATCH 40/85] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 797455816f0..068e08f6819 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -291,7 +291,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx const auto & column = result.getByPosition(result.columns() - 1).column; for (size_t i = 0; i < size; ++i) - if (column->getUInt(i) & 1) + if (!column->isNullAt(i) && (column->get64(i) & 1)) return true; return false; From 11af3fd54f6ee3ed0291fee9ed88a852f03a252a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 16:13:41 +0300 Subject: [PATCH 41/85] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 068e08f6819..3e5cbb34556 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -267,6 +267,8 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->removeUnusedActions(); actions = std::make_shared(filter_actions_dag); + + actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; } bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const @@ -288,7 +290,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx Block result = granule.block; actions->execute(result); - const auto & column = result.getByPosition(result.columns() - 1).column; + const auto & column = result.getByName(actions_output_column_name).column; for (size_t i = 0; i < size; ++i) if (!column->isNullAt(i) && (column->get64(i) & 1)) From a28309689f26e161dfbaa014bc51dea7460de30f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 May 2024 16:13:58 +0300 Subject: [PATCH 42/85] Update MergeTreeIndexSet.h --- src/Storages/MergeTree/MergeTreeIndexSet.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 3348b5fbe34..901653e47d6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -126,6 +126,7 @@ private: std::unordered_set key_columns; ExpressionActionsPtr actions; + String actions_output_column_name; }; From 5698ef698d20c12d83fa7f685cbfee9352b4583d Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 18 May 2024 21:17:22 +0200 Subject: [PATCH 43/85] check overall wf status in mergeable check --- .github/workflows/pull_request.yml | 18 +++++++----------- tests/ci/ci.py | 8 ++++---- tests/ci/commit_status_helper.py | 4 ++++ tests/ci/finish_check.py | 2 +- tests/ci/merge_pr.py | 15 ++++++++++++++- 5 files changed, 30 insertions(+), 17 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f2e4b5f328d..21c2e48677d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -140,13 +140,11 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should by run even if Builds_1/2 fail, so put them separatly in wf (not in Tests_1/2) + # Reports should by run even if Builds_1/2 fail, so put them separately in wf (not in Tests_1/2) Builds_1_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: - - RunConfig - - Builds_1 + if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} + needs: [RunConfig, StyleCheck, Builds_1] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check @@ -154,10 +152,8 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} Builds_2_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: - - RunConfig - - Builds_2 + if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} + needs: [RunConfig, StyleCheck, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse special build check @@ -165,7 +161,7 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} CheckReadyForMerge: - if: ${{ !cancelled() }} + if: ${{ !cancelled() && needs.StyleCheck.result == 'success' }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: @@ -176,7 +172,7 @@ jobs: - name: Check and set merge status run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 merge_pr.py --set-ci-status + python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} ################################# Stage Final ################################# # diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 3aa8f1bb813..3a616c8aad6 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -886,9 +886,9 @@ class CiOptions: for job in job_with_parents: if job in jobs_to_do and job not in jobs_to_do_requested: jobs_to_do_requested.append(job) - assert ( - jobs_to_do_requested - ), f"Include tags are set but no job configured - Invalid tags, probably [{self.include_keywords}]" + print( + f"WARNING: Include tags are set but no job configured - Invalid tags, probably [{self.include_keywords}]" + ) if JobNames.STYLE_CHECK not in jobs_to_do_requested: # Style check must not be omitted jobs_to_do_requested.append(JobNames.STYLE_CHECK) @@ -898,7 +898,7 @@ class CiOptions: if self.ci_sets: for tag in self.ci_sets: label_config = CI_CONFIG.get_label_config(tag) - assert label_config, f"Unknonwn tag [{tag}]" + assert label_config, f"Unknown tag [{tag}]" print( f"NOTE: CI Set's tag: [{tag}], add jobs: [{label_config.run_jobs}]" ) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index ec9746e9af9..733b07813a5 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -471,6 +471,7 @@ def trigger_mergeable_check( statuses: CommitStatuses, hide_url: bool = False, set_if_green: bool = False, + workflow_failed: bool = False, ) -> StatusType: """calculate and update StatusNames.MERGEABLE""" required_checks = [ @@ -501,6 +502,9 @@ def trigger_mergeable_check( if fail: description = "failed: " + ", ".join(fail) state = FAILURE + elif workflow_failed: + description = "check workflow failures" + state = FAILURE description = format_description(description) if not set_if_green and state == SUCCESS: diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 615b26b51f0..b31be7654d3 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -17,7 +17,7 @@ from get_robot_token import get_best_robot_token from pr_info import PRInfo from report import PENDING, SUCCESS from synchronizer_utils import SYNC_BRANCH_PREFIX -from tests.ci.env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY +from env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY def main(): diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 519fa5fcebb..500de4eb718 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -182,6 +182,12 @@ def parse_args() -> argparse.Namespace: action="store_true", help="if set, only update/set Mergeable Check status", ) + parser.add_argument( + "--wf-status", + type=str, + default="", + help="overall workflow status [success|failure]. used with --set-ci-status only", + ) parser.add_argument( "--check-approved", action="store_true", @@ -237,10 +243,17 @@ def main(): repo = gh.get_repo(args.repo) if args.set_ci_status: + assert args.wf_status in ("failure", "success") # set mergeable check status and exit commit = get_commit(gh, args.pr_info.sha) statuses = get_commit_filtered_statuses(commit) - trigger_mergeable_check(commit, statuses, hide_url=False, set_if_green=True) + trigger_mergeable_check( + commit, + statuses, + hide_url=False, + set_if_green=True, + workflow_failed=(args.wf_status != "success"), + ) return # An ugly and not nice fix to patch the wrong organization URL, From 83572b7f05e0ba68accd0ea766f243d7adbaf070 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 17:23:52 +0000 Subject: [PATCH 44/85] Trigger build From 4a8e663b1ae72b1d57ad15adce9eddbb3f6e1d28 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 15:45:23 +0000 Subject: [PATCH 45/85] Fix clang-tidy after clang-18 transition --- .clang-tidy | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index e2f318562ec..d4c57a52dd6 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -22,6 +22,7 @@ Checks: [ '-bugprone-exception-escape', '-bugprone-forward-declaration-namespace', '-bugprone-implicit-widening-of-multiplication-result', + '-bugprone-multi-level-implicit-pointer-conversion', '-bugprone-narrowing-conversions', '-bugprone-not-null-terminated-result', '-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged @@ -98,6 +99,7 @@ Checks: [ '-modernize-use-nodiscard', '-modernize-use-trailing-return-type', + '-performance-enum-size', '-performance-inefficient-string-concatenation', '-performance-no-int-to-ptr', '-performance-avoid-endl', @@ -105,6 +107,7 @@ Checks: [ '-portability-simd-intrinsics', + '-readability-avoid-nested-conditional-operator', '-readability-avoid-unconditional-preprocessor-if', '-readability-braces-around-statements', '-readability-convert-member-functions-to-static', @@ -118,6 +121,12 @@ Checks: [ '-readability-magic-numbers', '-readability-named-parameter', '-readability-redundant-declaration', + '-readability-redundant-inline-specifier', + '-readability-redundant-member-init', # useful but triggers another problem. Imagine a struct S with String members. If S is + # initialized via designated initializer, then the caller needs to initialize *all* members + # (to avoid warning `missing-field-initializers`) unless the mebers are default-initialized + # in S. So this clang-tidy warning prevents default-initialization and writing out all members + # in a designated initializer is ugly... '-readability-simplify-boolean-expr', '-readability-suspicious-call-argument', '-readability-uppercase-literal-suffix', @@ -125,17 +134,6 @@ Checks: [ '-zircon-*', - # These are new in clang-18, and we have to sort them out: - '-readability-avoid-nested-conditional-operator', - '-modernize-use-designated-initializers', - '-performance-enum-size', - '-readability-redundant-inline-specifier', - '-readability-redundant-member-init', - '-bugprone-crtp-constructor-accessibility', - '-bugprone-suspicious-stringview-data-usage', - '-bugprone-multi-level-implicit-pointer-conversion', - '-cert-err33-c', - # This is a good check, but clang-tidy crashes, see https://github.com/llvm/llvm-project/issues/91872 '-modernize-use-constraints', # https://github.com/abseil/abseil-cpp/issues/1667 From 56e8027883f7732cf090f835b6e25dd329307bc1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 17:53:06 +0000 Subject: [PATCH 46/85] Improve comment --- .clang-tidy | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index d4c57a52dd6..219ac263ab3 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -122,11 +122,11 @@ Checks: [ '-readability-named-parameter', '-readability-redundant-declaration', '-readability-redundant-inline-specifier', - '-readability-redundant-member-init', # useful but triggers another problem. Imagine a struct S with String members. If S is - # initialized via designated initializer, then the caller needs to initialize *all* members - # (to avoid warning `missing-field-initializers`) unless the mebers are default-initialized - # in S. So this clang-tidy warning prevents default-initialization and writing out all members - # in a designated initializer is ugly... + '-readability-redundant-member-init', # Useful but triggers another problem. Imagine a struct S with multiple String members. Structs are often instantiated via designated + # initializer S s{.s1 = [...], .s2 = [...], [...]}. In this case, compiler warning `missing-field-initializers` requires to specify all members which are not in-struct + # initialized (example: s1 in struct S { String s1; String s2{};}; is not in-struct initialized, therefore it must be specified at instantiation time). As explicitly + # specifying all members is tedious for large structs, `missing-field-initializers` makes programmers initialize as many members as possible in-struct. Clang-tidy + # warning `readability-redundant-member-init` does the opposite thing, both are not compatible with each other. '-readability-simplify-boolean-expr', '-readability-suspicious-call-argument', '-readability-uppercase-literal-suffix', From 2765fd951cbcd6f5c576ee1919ae644cb4d76256 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 19 May 2024 21:02:12 +0200 Subject: [PATCH 47/85] Properly support native copy for azure --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupIO_AzureBlobStorage.cpp | 38 ++-- src/Backups/BackupIO_AzureBlobStorage.h | 15 +- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 + src/Backups/BackupsWorker.cpp | 1 + .../registerBackupEngineAzureBlobStorage.cpp | 22 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 7 +- .../AzureBlobStorage/AzureObjectStorage.h | 6 +- .../ObjectStorages/ObjectStorageFactory.cpp | 5 +- .../copyAzureBlobStorageFile.cpp | 1 + src/Storages/StorageAzureBlob.cpp | 10 +- src/Storages/StorageAzureBlob.h | 2 + .../TableFunctionAzureBlobStorage.cpp | 4 +- .../TableFunctionAzureBlobStorageCluster.cpp | 4 +- .../__init__.py | 1 + .../test.py | 215 ++++++++++++++++++ 17 files changed, 301 insertions(+), 35 deletions(-) create mode 100644 tests/integration/test_azure_blob_storage_native_copy/__init__.py create mode 100644 tests/integration/test_azure_blob_storage_native_copy/test.py diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 4e752508577..e13a9a12ca2 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -39,6 +39,7 @@ public: std::optional backup_uuid; bool deduplicate_files = true; bool allow_s3_native_copy = true; + bool allow_azure_native_copy = true; bool use_same_s3_credentials_for_base_backup = false; bool azure_attempt_to_create_container = true; ReadSettings read_settings; diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index a3998431674..672a68e089f 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -31,22 +31,28 @@ namespace ErrorCodes BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( StorageAzureBlob::Configuration configuration_, + bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURLWithContainer(), false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); - object_storage = std::make_unique("BackupReaderAzureBlobStorage", - std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration_.container); + object_storage = std::make_unique( + "BackupReaderAzureBlobStorage", + std::move(client_ptr), + StorageAzureBlob::createSettings(context_), + configuration.container, + configuration.getConnectionURLWithContainer()); + client = object_storage->getAzureBlobStorageClient(); - settings = object_storage->getSettings(); + auto settings_copy = *object_storage->getSettings(); + settings_copy.use_native_copy = allow_azure_native_copy; + settings = std::make_unique(settings_copy); } BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; @@ -76,9 +82,9 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if ((destination_data_source_description.type == DataSourceType::ObjectStorage) - && (destination_data_source_description.object_storage_type == ObjectStorageType::Azure) - && (destination_data_source_description.is_encrypted == encrypted_in_backup)) + LOG_TRACE(log, "Source description {} desctionation description {}", data_source_description.description, destination_data_source_description.description); + if (destination_data_source_description.sameKind(data_source_description) + && destination_data_source_description.is_encrypted == encrypted_in_backup) { LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t @@ -116,12 +122,13 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( StorageAzureBlob::Configuration configuration_, + bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURLWithContainer(), false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); @@ -130,9 +137,12 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), StorageAzureBlob::createSettings(context_), - configuration_.container); + configuration_.container, + configuration.getConnectionURLWithContainer()); client = object_storage->getAzureBlobStorageClient(); - settings = object_storage->getSettings(); + auto settings_copy = *object_storage->getSettings(); + settings_copy.use_native_copy = allow_azure_native_copy; + settings = std::make_unique(settings_copy); } void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -140,7 +150,9 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu { /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + LOG_TRACE(log, "Source description {} desctionation description {}", source_data_source_description.description, data_source_description.description); + if (source_data_source_description.sameKind(data_source_description) + && source_data_source_description.is_encrypted == copy_encrypted) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage container. /// In this case we can't use the native copy. diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index f0b9aace4d4..3a909ab684a 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -16,7 +16,12 @@ namespace DB class BackupReaderAzureBlobStorage : public BackupReaderDefault { public: - BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + BackupReaderAzureBlobStorage( + StorageAzureBlob::Configuration configuration_, + bool allow_azure_native_copy, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_); ~BackupReaderAzureBlobStorage() override; bool fileExists(const String & file_name) override; @@ -37,7 +42,13 @@ private: class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: - BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container); + BackupWriterAzureBlobStorage( + StorageAzureBlob::Configuration configuration_, + bool allow_azure_native_copy, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_, + bool attempt_to_create_container); ~BackupWriterAzureBlobStorage() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 06f49dfa448..e33880f88e3 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ M(Bool, allow_s3_native_copy) \ + M(Bool, allow_azure_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ M(Bool, azure_attempt_to_create_container) \ M(Bool, read_from_filesystem_cache) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index eccf4e90ce7..a6c4d5d7181 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -44,6 +44,9 @@ struct BackupSettings /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) bool allow_s3_native_copy = true; + /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) + bool allow_azure_native_copy = true; + /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 9057dc9d198..69d9c52ebd9 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -598,6 +598,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; + backup_create_params.allow_azure_native_copy = backup_settings.allow_azure_native_copy; backup_create_params.use_same_s3_credentials_for_base_backup = backup_settings.use_same_s3_credentials_for_base_backup; backup_create_params.azure_attempt_to_create_container = backup_settings.azure_attempt_to_create_container; backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings); diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 1b9545fc455..8b05965f472 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -135,10 +135,12 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(configuration, - params.read_settings, - params.write_settings, - params.context); + auto reader = std::make_shared( + configuration, + params.allow_azure_native_copy, + params.read_settings, + params.write_settings, + params.context); return std::make_unique( params.backup_info, @@ -150,11 +152,13 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } else { - auto writer = std::make_shared(configuration, - params.read_settings, - params.write_settings, - params.context, - params.azure_attempt_to_create_container); + auto writer = std::make_shared( + configuration, + params.allow_azure_native_copy, + params.read_settings, + params.write_settings, + params.context, + params.azure_attempt_to_create_container); return std::make_unique( params.backup_info, diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 36225b13ee8..bee8e206ec4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -107,11 +107,13 @@ AzureObjectStorage::AzureObjectStorage( const String & name_, AzureClientPtr && client_, SettingsPtr && settings_, - const String & object_namespace_) + const String & object_namespace_, + const String & description_) : name(name_) , client(std::move(client_)) , settings(std::move(settings_)) , object_namespace(object_namespace_) + , description(description_) , log(getLogger("AzureObjectStorage")) { } @@ -409,7 +411,8 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - object_namespace + object_namespace, + description ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f52ab803012..3d94090bcc6 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -81,7 +81,8 @@ public: const String & name_, AzureClientPtr && client_, SettingsPtr && settings_, - const String & object_namespace_); + const String & object_namespace_, + const String & description_); void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; @@ -93,7 +94,7 @@ public: std::string getCommonKeyPrefix() const override { return ""; } - std::string getDescription() const override { return client.get()->GetUrl(); } + std::string getDescription() const override { return description; } bool exists(const StoredObject & object) const override; @@ -172,6 +173,7 @@ private: MultiVersion client; MultiVersion settings; const String object_namespace; /// container + prefix + const String description; /// url + container LoggerPtr log; }; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 761ff24e648..cddcea979b5 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -306,11 +306,14 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) bool /* skip_access_check */) -> ObjectStoragePtr { AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix); + std::string endpoint_string = endpoint.getEndpoint(); + return createObjectStorage( ObjectStorageType::Azure, config, config_prefix, name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix); + endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix, + endpoint.prefix.empty() ? endpoint_string : endpoint_string.substr(0, endpoint_string.length() - endpoint.prefix.length())); }; factory.registerObjectStorageType("azure_blob_storage", creator); factory.registerObjectStorageType("azure", creator); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 769f1a184f6..dc46de1e07f 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -289,6 +289,7 @@ void copyAzureBlobStorageFile( if (settings->use_native_copy) { + LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob); ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (dest_client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 9c551e82a99..0103fc0d2a2 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -302,8 +302,8 @@ void registerStorageAzureBlob(StorageFactory & factory) auto settings = StorageAzureBlob::createSettings(args.getContext()); return std::make_shared( - std::move(configuration), - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings),configuration.container), + configuration, + std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), args.getContext(), args.table_id, args.columns, @@ -491,6 +491,12 @@ Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); } +std::string StorageAzureBlob::Configuration::getConnectionURLWithContainer() const +{ + auto url = getConnectionURL(); + return fs::path(url.toString()) / container; +} + bool StorageAzureBlob::Configuration::withGlobsIgnorePartitionWildcard() const { if (!withPartitionWildcard()) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index b433cd92d68..7bce40bce26 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -45,6 +45,8 @@ public: Poco::URI getConnectionURL() const; + std::string getConnectionURLWithContainer() const; + std::string connection_url; bool is_connection_string; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 275cd2a9cbb..e73277b4d7b 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -333,7 +333,7 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container); + auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()); if (configuration.format == "auto") return StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, std::nullopt, context).first; return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); @@ -365,7 +365,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct StoragePtr storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), context, StorageID(getDatabaseName(), table_name), columns, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index 04dddca7672..dc65426a6e3 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -39,7 +39,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( /// On worker node this filename won't contains globs storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), context, StorageID(getDatabaseName(), table_name), columns, @@ -54,7 +54,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( storage = std::make_shared( cluster_name, configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, diff --git a/tests/integration/test_azure_blob_storage_native_copy/__init__.py b/tests/integration/test_azure_blob_storage_native_copy/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_azure_blob_storage_native_copy/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_azure_blob_storage_native_copy/test.py b/tests/integration/test_azure_blob_storage_native_copy/test.py new file mode 100644 index 00000000000..b16d9b4b5c4 --- /dev/null +++ b/tests/integration/test_azure_blob_storage_native_copy/test.py @@ -0,0 +1,215 @@ +#!/usr/bin/env python3 + +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +from azure.storage.blob import BlobServiceClient +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry + + +def generate_config(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + TEMPLATE = """ + + + + + local + object_storage + azure_blob_storage + http://azurite1:{port}/devstoreaccount1 + cont + false + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + true + + + local + object_storage + azure_blob_storage + true + http://azurite1:{port}/devstoreaccount1 + othercontainer + false + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + cache + disk_azure + /tmp/azure_cache/ + 1000000000 + 1 + + + + + +
+ disk_azure +
+
+
+ + +
+ disk_azure_other_bucket +
+
+
+ + +
+ disk_azure_cache +
+
+
+
+
+ + disk_azure + disk_azure_cache + disk_azure_other_bucket + +
+ """ + f.write(TEMPLATE.format(port=port)) + return path + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_config(port) + cluster.add_instance( + "node1", + main_configs=[path], + with_azurite=True, + ) + cluster.add_instance( + "node2", + main_configs=[path], + with_azurite=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def azure_query( + node, query, expect_error=False, try_num=10, settings={}, query_on_retry=None +): + for i in range(try_num): + try: + if expect_error: + return node.query_and_get_error(query, settings=settings) + else: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) + break + if not retry or i == try_num - 1: + raise Exception(ex) + if query_on_retry is not None: + node.query(query_on_retry) + continue + + +def test_backup_restore_on_merge_tree_same_container(cluster): + node1 = cluster.instances["node1"] + azure_query( + node1, + f"CREATE TABLE test_simple_merge_tree(key UInt64, data String) Engine = MergeTree() ORDER BY tuple() SETTINGS storage_policy='policy_azure_cache'", + ) + azure_query(node1, f"INSERT INTO test_simple_merge_tree VALUES (1, 'a')") + + backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_merge_tree_backup')" + print("BACKUP DEST", backup_destination) + azure_query( + node1, + f"BACKUP TABLE test_simple_merge_tree TO {backup_destination}", + ) + + assert node1.contains_in_log("using native copy") + + azure_query( + node1, + f"RESTORE TABLE test_simple_merge_tree AS test_simple_merge_tree_restored FROM {backup_destination};", + ) + assert ( + azure_query(node1, f"SELECT * from test_simple_merge_tree_restored") == "1\ta\n" + ) + azure_query(node1, f"DROP TABLE test_simple_merge_tree") + azure_query(node1, f"DROP TABLE test_simple_merge_tree_restored") + + +def test_backup_restore_on_merge_tree_different_container(cluster): + node2 = cluster.instances["node2"] + azure_query( + node2, + f"CREATE TABLE test_simple_merge_tree_different_bucket(key UInt64, data String) Engine = MergeTree() ORDER BY tuple() SETTINGS storage_policy='policy_azure_other_bucket'", + ) + azure_query( + node2, f"INSERT INTO test_simple_merge_tree_different_bucket VALUES (1, 'a')" + ) + + backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_merge_tree_different_bucket_backup_different_bucket')" + print("BACKUP DEST", backup_destination) + azure_query( + node2, + f"BACKUP TABLE test_simple_merge_tree_different_bucket TO {backup_destination}", + ) + + assert not node2.contains_in_log("using native copy") + + azure_query( + node2, + f"RESTORE TABLE test_simple_merge_tree_different_bucket AS test_simple_merge_tree_different_bucket_restored FROM {backup_destination};", + ) + assert ( + azure_query( + node2, f"SELECT * from test_simple_merge_tree_different_bucket_restored" + ) + == "1\ta\n" + ) + + assert not node2.contains_in_log("using native copy") + + azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket") + azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket_restored") From 08091006aa66ae8b01b56b83ed9fe74482c2672d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 20:36:08 +0000 Subject: [PATCH 48/85] Ignore exception when checking for cgroupsv2 --- base/base/cgroupsv2.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index bea2e99fa51..5c5ab150564 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -9,11 +9,18 @@ bool cgroupsV2Enabled() { #if defined(OS_LINUX) - /// This file exists iff the host has cgroups v2 enabled. - auto controllers_file = default_cgroups_mount / "cgroup.controllers"; - if (!std::filesystem::exists(controllers_file)) - return false; - return true; + try + { + /// This file exists iff the host has cgroups v2 enabled. + auto controllers_file = default_cgroups_mount / "cgroup.controllers"; + if (!std::filesystem::exists(controllers_file)) + return false; + return true; + } + catch (...) + { + return false; /// e.g. permission denied exception + } #else return false; #endif From 63f31d3c1e2f4f562aebfe1c7a7cd26abad5ab1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 06:13:06 +0200 Subject: [PATCH 49/85] Add retries in `git submodule update` --- docker/test/fasttest/run.sh | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index bc7ffd1c2ef..4d5159cfa9e 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -160,10 +160,17 @@ function clone_submodules git submodule sync git submodule init - # --jobs does not work as fast as real parallel running - printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ - xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ - git submodule update --depth 1 --single-branch + + # Network is unreliable + for _ in {1..10} + do + # --jobs does not work as fast as real parallel running + printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ + xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ + git submodule update --depth 1 --single-branch && break + sleep 1 + done + git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd From ea303e0d6346cd1bdd96c87ef767856db9425133 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 09:54:20 +0300 Subject: [PATCH 50/85] HDFS is unsupported --- docs/en/operations/storing-data.md | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 7005783dd60..59de4989941 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -7,27 +7,27 @@ title: "External Disks for Storing Data" Data, processed in ClickHouse, is usually stored in the local file system — on the same machine with the ClickHouse server. That requires large-capacity disks, which can be expensive enough. To avoid that you can store the data remotely. Various storages are supported: 1. [Amazon S3](https://aws.amazon.com/s3/) object storage. -2. The Hadoop Distributed File System ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)) -3. [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). +2. [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). +3. Unsupported: The Hadoop Distributed File System ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)) :::note ClickHouse also has support for external table engines, which are different from external storage option described on this page as they allow to read data stored in some general file format (like Parquet), while on this page we are describing storage configuration for ClickHouse `MergeTree` family or `Log` family tables. 1. to work with data stored on `Amazon S3` disks, use [S3](/docs/en/engines/table-engines/integrations/s3.md) table engine. -2. to work with data in the Hadoop Distributed File System — [HDFS](/docs/en/engines/table-engines/integrations/hdfs.md) table engine. -3. to work with data stored in Azure Blob Storage use [AzureBlobStorage](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) table engine. +2. to work with data stored in Azure Blob Storage use [AzureBlobStorage](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) table engine. +3. Unsupported: to work with data in the Hadoop Distributed File System — [HDFS](/docs/en/engines/table-engines/integrations/hdfs.md) table engine. ::: ## Configuring external storage {#configuring-external-storage} -[MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) family table engines can store data to `S3`, `AzureBlobStorage`, `HDFS` using a disk with types `s3`, `azure_blob_storage`, `hdfs` accordingly. +[MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) family table engines can store data to `S3`, `AzureBlobStorage`, `HDFS` (unsupported) using a disk with types `s3`, `azure_blob_storage`, `hdfs` (unsupported) accordingly. Disk configuration requires: -1. `type` section, equal to one of `s3`, `azure_blob_storage`, `hdfs`, `local_blob_storage`, `web`. +1. `type` section, equal to one of `s3`, `azure_blob_storage`, `hdfs` (unsupported), `local_blob_storage`, `web`. 2. Configuration of a specific external storage type. Starting from 24.1 clickhouse version, it is possible to use a new configuration option. It requires to specify: 1. `type` equal to `object_storage` -2. `object_storage_type`, equal to one of `s3`, `azure_blob_storage` (or just `azure` from `24.3`), `hdfs`, `local_blob_storage` (or just `local` from `24.3`), `web`. +2. `object_storage_type`, equal to one of `s3`, `azure_blob_storage` (or just `azure` from `24.3`), `hdfs` (unsupported), `local_blob_storage` (or just `local` from `24.3`), `web`. Optionally, `metadata_type` can be specified (it is equal to `local` by default), but it can also be set to `plain`, `web` and, starting from `24.4`, `plain_rewritable`. Usage of `plain` metadata type is described in [plain storage section](/docs/en/operations/storing-data.md/#storing-data-on-webserver), `web` metadata type can be used only with `web` object storage type, `local` metadata type stores metadata files locally (each metadata files contains mapping to files in object storage and some additional meta information about them). @@ -328,7 +328,7 @@ Configuration: ``` -Starting from `24.1` it is possible configure any object storage disk (`s3`, `azure`, `hdfs`, `local`) using `plain` metadata type. +Starting from `24.1` it is possible configure any object storage disk (`s3`, `azure`, `hdfs` (unsupported), `local`) using `plain` metadata type. Configuration: ``` xml @@ -428,12 +428,14 @@ Examples of working configurations can be found in integration tests directory ( Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: -## Using HDFS storage {#hdfs-storage} +## Using HDFS storage (Unsupported) In this sample configuration: -- the disk is of type `hdfs` +- the disk is of type `hdfs` (unsupported) - the data is hosted at `hdfs://hdfs1:9000/clickhouse/` +By the way, HDFS is unsupported and won't work. + ```xml @@ -464,9 +466,11 @@ In this sample configuration: ``` +Keep in mind that HDFS does not work at all. + ### Using Data Encryption {#encrypted-virtual-file-system} -You can encrypt the data stored on [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. +You can encrypt the data stored on [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) (unsupported) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. Example of disk configuration: @@ -529,7 +533,7 @@ Example of disk configuration: It is possible to configure local cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. -For versions >= 23.5 cache is supported only for remote disk types: S3, Azure, HDFS. +For versions >= 23.5 cache is supported only for remote disk types: S3, Azure, HDFS (unsupported). Cache uses `LRU` cache policy. @@ -971,7 +975,7 @@ Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#htt ### Zero-copy Replication (not ready for production) {#zero-copy} -Zero-copy replication is possible, but not recommended, with `S3` and `HDFS` disks. Zero-copy replication means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. +Zero-copy replication is possible, but not recommended, with `S3` and `HDFS` (unsupported) disks. Zero-copy replication means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. :::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. From f7c9fa696f91c306550ed3435bb51999983903e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 09:45:23 +0200 Subject: [PATCH 51/85] Fix UBSan error in negative positional arguments --- src/Interpreters/replaceForPositionalArguments.cpp | 2 +- .../03157_negative_positional_arguments_ubsan.reference | 0 .../0_stateless/03157_negative_positional_arguments_ubsan.sql | 1 + 3 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.reference create mode 100644 tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.sql diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index cceb0650fcd..3d60723a167 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -44,7 +44,7 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel pos = value; else { - if (static_cast(std::abs(value)) > columns.size()) + if (value < -static_cast(columns.size())) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Negative positional argument number {} is out of bounds. Expected in range [-{}, -1]", diff --git a/tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.reference b/tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.sql b/tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.sql new file mode 100644 index 00000000000..ddf5185c945 --- /dev/null +++ b/tests/queries/0_stateless/03157_negative_positional_arguments_ubsan.sql @@ -0,0 +1 @@ +SELECT 1 GROUP BY -9223372036854775808; -- { serverError BAD_ARGUMENTS } From 95d680e20423490e74bed25bbd865a6d033dbee8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 May 2024 10:09:57 +0200 Subject: [PATCH 52/85] Remove useless test --- tests/queries/0_stateless/00694_max_block_size_zero.reference | 0 tests/queries/0_stateless/00694_max_block_size_zero.sql | 4 ---- 2 files changed, 4 deletions(-) delete mode 100644 tests/queries/0_stateless/00694_max_block_size_zero.reference delete mode 100644 tests/queries/0_stateless/00694_max_block_size_zero.sql diff --git a/tests/queries/0_stateless/00694_max_block_size_zero.reference b/tests/queries/0_stateless/00694_max_block_size_zero.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/00694_max_block_size_zero.sql b/tests/queries/0_stateless/00694_max_block_size_zero.sql deleted file mode 100644 index ba5b513bb5d..00000000000 --- a/tests/queries/0_stateless/00694_max_block_size_zero.sql +++ /dev/null @@ -1,4 +0,0 @@ -SET send_logs_level = 'fatal'; - -SET max_block_size = 0; -SELECT number FROM system.numbers; -- { serverError 12 } From 1525ca4cf02479e53ebb670720ea385ddb7670a1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 May 2024 12:09:47 +0200 Subject: [PATCH 53/85] Disable pretty format restrictions when stdout is not TTY Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 4 ++-- .../queries/0_stateless/03160_pretty_format_tty.reference | 1 + tests/queries/0_stateless/03160_pretty_format_tty.sh | 8 ++++++++ 3 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03160_pretty_format_tty.reference create mode 100755 tests/queries/0_stateless/03160_pretty_format_tty.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 67aba2256e8..4441d884754 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -710,8 +710,8 @@ void ClientBase::adjustSettings() settings.input_format_values_allow_data_after_semicolon.changed = false; } - /// Do not limit pretty format output in case of --pager specified. - if (!pager.empty()) + /// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty. + if (!pager.empty() || !stdout_is_a_tty) { if (!global_context->getSettingsRef().output_format_pretty_max_rows.changed) { diff --git a/tests/queries/0_stateless/03160_pretty_format_tty.reference b/tests/queries/0_stateless/03160_pretty_format_tty.reference new file mode 100644 index 00000000000..6a5b453966d --- /dev/null +++ b/tests/queries/0_stateless/03160_pretty_format_tty.reference @@ -0,0 +1 @@ +100004 diff --git a/tests/queries/0_stateless/03160_pretty_format_tty.sh b/tests/queries/0_stateless/03160_pretty_format_tty.sh new file mode 100755 index 00000000000..bbc4b96eb90 --- /dev/null +++ b/tests/queries/0_stateless/03160_pretty_format_tty.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 + +# default output_format_pretty_max_rows is 10K +$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1" | wc -l From 434816820d31e09873453024e9563fca06b5b3d4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 18 May 2024 15:27:13 +0200 Subject: [PATCH 54/85] Fix upgrade settings changes check settings changes check simply checks that the settings are either the same, or there is a record in system.settings_changes, however, the problem is that clickhouse-local now adjusts some settings for Pretty format in case of stdout is not a tty, and this is the case for this check. So to avoid this, just run the clickhouse-local under script(1) to fool it. Signed-off-by: Azat Khuzhin --- docker/test/upgrade/run.sh | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 6761ddba3e5..29174cc87e6 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -58,8 +58,14 @@ echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/sys # Install previous release packages install_packages previous_release_package_folder -# Save old settings from system table for settings changes check -clickhouse-local -q "select * from system.settings format Native" > old_settings.native +# NOTE: we need to run clickhouse-local under script to get settings without any adjustments, like clickhouse-local does in case of stdout is not a tty +function save_settings_clean() +{ + local out=$1 && shift + script -q -c "clickhouse-local -q \"select * from system.settings into outfile '$out'\"" --log-out /dev/null +} + +save_settings_clean 'old_settings.native' # Initial run without S3 to create system.*_log on local file system to make it # available for dump via clickhouse-local @@ -183,7 +189,7 @@ configure IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'") if [ "${IS_SANITIZED}" -eq "0" ] then - clickhouse-local -q "select * from system.settings format Native" > new_settings.native + save_settings_clean 'new_settings.native' clickhouse-local -nmq " CREATE TABLE old_settings AS file('old_settings.native'); CREATE TABLE new_settings AS file('new_settings.native'); From 925ed89195008a787eeebd9213ac59f1a8adb17b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 10:31:32 +0200 Subject: [PATCH 55/85] More instrumentation around index --- src/Common/CurrentMetrics.cpp | 6 ++++-- src/Databases/DatabaseOnDisk.cpp | 3 +++ ...ObjectStorageRemoteMetadataRestoreHelper.cpp | 1 + src/Interpreters/AsynchronousInsertQueue.cpp | 3 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 17 ++++++++++++++++- src/Storages/System/StorageSystemReplicas.cpp | 2 ++ 6 files changed, 28 insertions(+), 4 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index b9916130bb9..21b4d114d79 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -288,8 +288,10 @@ M(HTTPConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for http hosts") \ \ M(AddressesActive, "Total count of addresses which are used for creation connections with connection pools") \ - M(AddressesBanned, "Total count of addresses which are banned as faulty for creation connections with connection pools") \ - + M(AddressesBanned, "Total count of addresses which are banned as faulty for creation connections with connection pools") \ + \ + M(FilteringMarksWithPrimaryKey, "Number of threads currently doing filtering of mark ranges by the primary key") \ + M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 5b9723fabc5..161be35f129 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -26,6 +26,8 @@ #include #include #include +#include + namespace fs = std::filesystem; @@ -665,6 +667,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat pool.scheduleOrThrowOnError( [batch, &process_metadata_file, &process_tmp_drop_metadata_file]() mutable { + setThreadName("DatabaseOnDisk"); for (const auto & file : batch) if (file.second) process_metadata_file(file.first); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 0314e0a7e92..18a0377efe7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -129,6 +129,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchemaRecu { pool.scheduleOrThrowOnError([this, path] { + setThreadName("BackupWorker"); for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) migrateFileToRestorableSchema(it->path()); }); diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index ab29c64184d..d72f3d81549 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -695,7 +695,6 @@ String serializeQuery(const IAST & query, size_t max_length) } -// static void AsynchronousInsertQueue::processData( InsertQuery key, InsertDataPtr data, ContextPtr global_context, QueueShardFlushTimeHistory & queue_shard_flush_time_history) try @@ -705,6 +704,8 @@ try SCOPE_EXIT(CurrentMetrics::sub(CurrentMetrics::PendingAsyncInsert, data->entries.size())); + setThreadName("AsyncInsertQ"); + const auto log = getLogger("AsynchronousInsertQueue"); const auto & insert_query = assert_cast(*key.query); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index de769c59d33..2b1b1b26347 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -53,6 +53,8 @@ namespace CurrentMetrics extern const Metric MergeTreeDataSelectExecutorThreads; extern const Metric MergeTreeDataSelectExecutorThreadsActive; extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; + extern const Metric FilteringMarksWithPrimaryKey; + extern const Metric FilteringMarksWithSecondaryKeys; } namespace DB @@ -664,15 +666,22 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); if (metadata_snapshot->hasPrimaryKey() || part_offset_condition) + { + CurrentMetrics::Increment metric(CurrentMetrics::FilteringMarksWithPrimaryKey); ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, part_offset_condition, settings, log); + } else if (total_marks_count) + { ranges.ranges = MarkRanges{{MarkRange{0, total_marks_count}}}; + } sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); if (!ranges.ranges.empty()) sum_parts_pk.fetch_add(1, std::memory_order_relaxed); + CurrentMetrics::Increment metric(CurrentMetrics::FilteringMarksWithSecondaryKeys); + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { if (ranges.ranges.empty()) @@ -733,6 +742,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd num_threads = std::min(num_streams, settings.max_threads_for_indexes); } + LOG_TRACE(log, "Filtering marks by primary and secondary keys"); + if (num_threads <= 1) { for (size_t part_index = 0; part_index < parts.size(); ++part_index) @@ -740,7 +751,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } else { - /// Parallel loading of data parts. + /// Parallel loading and filtering of data parts. ThreadPool pool( CurrentMetrics::MergeTreeDataSelectExecutorThreads, CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, @@ -748,8 +759,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd num_threads); for (size_t part_index = 0; part_index < parts.size(); ++part_index) + { pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] { + setThreadName("MergeTreeIndex"); + SCOPE_EXIT_SAFE( if (thread_group) CurrentThread::detachFromGroupIfNotDetached(); @@ -759,6 +773,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd process_part(part_index); }); + } pool.wait(); } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 5045dec3682..10d5c353c43 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -141,6 +141,8 @@ public: if (thread_group) CurrentThread::attachToGroupIfDetached(thread_group); + setThreadName("SystemReplicas"); + try { ReplicatedTableStatus status; From 314752e044bcdf5d7516d5188e7e6a4302467b90 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 May 2024 11:55:44 +0200 Subject: [PATCH 56/85] Update src/Backups/BackupIO_AzureBlobStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Backups/BackupIO_AzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 672a68e089f..5f6495e5733 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -82,7 +82,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { auto destination_data_source_description = destination_disk->getDataSourceDescription(); - LOG_TRACE(log, "Source description {} desctionation description {}", data_source_description.description, destination_data_source_description.description); + LOG_TRACE(log, "Source description {}, desctionation description {}", data_source_description.description, destination_data_source_description.description); if (destination_data_source_description.sameKind(data_source_description) && destination_data_source_description.is_encrypted == encrypted_in_backup) { From 6a7a09a1f902ec4fb54cd19f9368581cddf6da1e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 May 2024 11:55:50 +0200 Subject: [PATCH 57/85] Update src/Backups/BackupIO_AzureBlobStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Backups/BackupIO_AzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 5f6495e5733..b035125a545 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -150,7 +150,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu { /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. auto source_data_source_description = src_disk->getDataSourceDescription(); - LOG_TRACE(log, "Source description {} desctionation description {}", source_data_source_description.description, data_source_description.description); + LOG_TRACE(log, "Source description {}, desctionation description {}", source_data_source_description.description, data_source_description.description); if (source_data_source_description.sameKind(data_source_description) && source_data_source_description.is_encrypted == copy_encrypted) { From 23e87ef80ee332b71d5e9a1d51e81de9d4626a84 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 May 2024 11:56:44 +0200 Subject: [PATCH 58/85] Review fix --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index dc46de1e07f..667e63729ca 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -289,7 +289,7 @@ void copyAzureBlobStorageFile( if (settings->use_native_copy) { - LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob); + LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob); ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (dest_client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); From e959ba9578337b30fe6765ab4d9bcf03fb7668a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 13:38:19 +0300 Subject: [PATCH 59/85] Update docs/en/operations/storing-data.md Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- docs/en/operations/storing-data.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 59de4989941..e9370c02463 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -434,7 +434,7 @@ In this sample configuration: - the disk is of type `hdfs` (unsupported) - the data is hosted at `hdfs://hdfs1:9000/clickhouse/` -By the way, HDFS is unsupported and won't work. +By the way, HDFS is unsupported and therefore there might be issues when using it. Feel free to make a pull request with the fix if any issue arises. ```xml From 7d3a601eccef2094f60b61cf295a6f88d9fbd0f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 13:39:07 +0300 Subject: [PATCH 60/85] Update docs/en/operations/storing-data.md --- docs/en/operations/storing-data.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index e9370c02463..9b316960750 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -466,7 +466,7 @@ By the way, HDFS is unsupported and therefore there might be issues when using i ``` -Keep in mind that HDFS does not work at all. +Keep in mind that HDFS may not work in corner cases. ### Using Data Encryption {#encrypted-virtual-file-system} From d32c059e7f83830cc8654ddc24e55008363df113 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 May 2024 13:48:14 +0200 Subject: [PATCH 61/85] Move changes from private --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 13 +++++++++---- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 463ca07ec57..4a34f828801 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -793,7 +793,8 @@ void IMergeTreeDataPart::addProjectionPart( projection_parts[projection_name] = std::move(projection_part); } -void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency, bool & has_broken_projection, bool if_not_loaded) +void IMergeTreeDataPart::loadProjections( + bool require_columns_checksums, bool check_consistency, bool & has_broken_projection, bool if_not_loaded, bool only_metadata) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); for (const auto & projection : metadata_snapshot->projections) @@ -813,7 +814,10 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch try { - part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); + if (only_metadata) + part->loadChecksums(require_columns_checksums); + else + part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); } catch (...) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c380f99060e..9ee01c0efc4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -445,7 +445,15 @@ public: bool hasBrokenProjection(const String & projection_name) const; /// Return true, if all projections were loaded successfully and none was marked as broken. - void loadProjections(bool require_columns_checksums, bool check_consistency, bool & has_broken_projection, bool if_not_loaded = false); + void loadProjections( + bool require_columns_checksums, + bool check_consistency, + bool & has_broken_projection, + bool if_not_loaded = false, + bool only_metadata = false); + + /// If checksums.txt exists, reads file's checksums (and sizes) from it + void loadChecksums(bool require); void setBrokenReason(const String & message, int code) const; @@ -671,9 +679,6 @@ private: static void appendFilesOfColumns(Strings & files); - /// If checksums.txt exists, reads file's checksums (and sizes) from it - void loadChecksums(bool require); - static void appendFilesOfChecksums(Strings & files); /// Loads marks index granularity into memory From 6f56642567cea3867a57f1f4c96c4c3a35be1ed7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 20 May 2024 13:13:50 +0000 Subject: [PATCH 62/85] Incorporate review feedback --- base/base/cgroupsv2.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index 5c5ab150564..f20b9daf22e 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -17,9 +17,9 @@ bool cgroupsV2Enabled() return false; return true; } - catch (...) + catch (const std::filesystem::filesystem_error &) /// all "underlying OS API errors", typically: permission denied { - return false; /// e.g. permission denied exception + return false; /// not logging the exception as most callers fall back to cgroups v1 } #else return false; From c4f47b907182e58c12dcf7934e4745ad8048788d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 20 May 2024 14:09:17 +0000 Subject: [PATCH 63/85] set greem mergeable status in upstream pr from finish check in sync --- tests/ci/commit_status_helper.py | 2 ++ tests/ci/finish_check.py | 8 +++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 733b07813a5..34e2d9f8085 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -522,6 +522,7 @@ def update_upstream_sync_status( sync_pr_number: int, gh: Github, state: StatusType, + can_set_green_mergeable_status: bool = False, ) -> None: upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) upstream_pr = upstream_repo.get_pull(upstream_pr_number) @@ -571,4 +572,5 @@ def update_upstream_sync_status( upstream_commit, get_commit_filtered_statuses(upstream_commit), True, + set_if_green=can_set_green_mergeable_status, ) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index b31be7654d3..1a7000f5353 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -51,7 +51,13 @@ def main(): and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY ): upstream_pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) - update_upstream_sync_status(upstream_pr_number, pr_info.number, gh, state) + update_upstream_sync_status( + upstream_pr_number, + pr_info.number, + gh, + state, + can_set_green_mergeable_status=True, + ) statuses = [s for s in statuses if s.context == StatusNames.CI] if not statuses: From fb619bac2f664fc0fa337fae3174332d397ce7f3 Mon Sep 17 00:00:00 2001 From: Maciej Bak Date: Mon, 20 May 2024 16:35:06 +0200 Subject: [PATCH 64/85] Fix backup all documentation --- 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 2ba50b39934..46c24ad8491 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -22,7 +22,7 @@ description: In order to effectively mitigate possible human errors, you should TEMPORARY TABLE table_name [AS table_name_in_backup] | VIEW view_name [AS view_name_in_backup] ALL TEMPORARY TABLES [EXCEPT ...] | - ALL DATABASES [EXCEPT ...] } [,...] + ALL [EXCEPT ...] } [,...] [ON CLUSTER 'cluster_name'] TO|FROM File('/') | Disk('', '/') | S3('/', '', '') [SETTINGS base_backup = File('/') | Disk(...) | S3('/', '', '')] From cb912a656e5245203b85d47fe6f6b0674319fd02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 17:07:35 +0200 Subject: [PATCH 65/85] Losen build resource limits for unusual architectures --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index abbc48ab23a..afea22a1b90 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,8 +61,8 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # set CPU time limit to 1000 seconds set (RLIMIT_CPU 1000) - # Sanitizers are too heavy - if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE) + # Sanitizers are too heavy. Some architectures too. + if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64 OR ARCH_LOONGARCH64) set (RLIMIT_DATA 10000000000) # 10G endif() From 85998672f01ab17f3dff9d8f430d28204d4de28c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 13 May 2024 10:21:03 +0200 Subject: [PATCH 66/85] tests: attempt to fix 02340_parts_refcnt_mergetree flakiness I'm not sure that this was the case, but I don't see any other reasons for now. I've also looked at the logs and trace_log, and did not noticed anything interesting. Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 208a9038681..0a96cc0e774 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -9,7 +9,11 @@ function check_refcnt_for_table() { local table=$1 && shift - $CLICKHOUSE_CLIENT -q "system stop merges $table" + $CLICKHOUSE_CLIENT -nm -q " + system stop merges $table; + -- cleanup thread may hold the parts lock + system stop cleanup $table; + " $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" local query_id From 0832f961070ea8e2ffd294a577036f8000ec49bf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 May 2024 14:30:25 +0200 Subject: [PATCH 67/85] tests: 02340_parts_refcnt_mergetree cleanup Signed-off-by: Azat Khuzhin --- .../0_stateless/02340_parts_refcnt_mergetree.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 0a96cc0e774..2d4e306cb3c 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -19,13 +19,19 @@ function check_refcnt_for_table() local query_id query_id="$table-$(random_str 10)" - SETTINGS="--format Null --max_threads 1 --max_block_size 1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.0" + local args=( + --format Null + --max_threads 1 + --max_block_size 1 + --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.0 + --query_id "$query_id" + ) # Notes: # - query may sleep 1*(200/4)=50 seconds maximum, it is enough to check system.parts # - "part = 1" condition should prune all parts except first # - max_block_size=1 with index_granularity=1 will allow to cancel the query earlier - $CLICKHOUSE_CLIENT $SETTINGS --query_id "$query_id" -q "select sleepEachRow(1) from $table where part = 1" & + $CLICKHOUSE_CLIENT "${args[@]}" -q "select sleepEachRow(1) from $table where part = 1" & PID=$! # wait for query to be started From 722b7ab5229c6e0267b6941363dd446220f337d0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 May 2024 14:43:42 +0200 Subject: [PATCH 68/85] tests: improve 02340_parts_refcnt_mergetree (from 12sec to 1sec) Signed-off-by: Azat Khuzhin --- .../02340_parts_refcnt_mergetree.sh | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 2d4e306cb3c..5fae571e217 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -19,33 +19,33 @@ function check_refcnt_for_table() local query_id query_id="$table-$(random_str 10)" + local log_file + log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log") local args=( --format Null --max_threads 1 --max_block_size 1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.0 --query_id "$query_id" + --send_logs_level "test" + --server_logs_file "$log_file" ) # Notes: - # - query may sleep 1*(200/4)=50 seconds maximum, it is enough to check system.parts + # - query may sleep 0.1*(200/4)=5 seconds maximum, it is enough to check system.parts # - "part = 1" condition should prune all parts except first # - max_block_size=1 with index_granularity=1 will allow to cancel the query earlier - $CLICKHOUSE_CLIENT "${args[@]}" -q "select sleepEachRow(1) from $table where part = 1" & + $CLICKHOUSE_CLIENT "${args[@]}" -q "select sleepEachRow(0.1) from $table where part = 1" & PID=$! - # wait for query to be started - while [ "$($CLICKHOUSE_CLIENT -q "select count() from system.processes where query_id = '$query_id'")" -ne 1 ]; do - sleep 0.1 - done - # When the query only starts it execution it holds reference for each part, # however when it starts reading, partition pruning takes place, # and it should hold only parts that are required for SELECT # - # But to reach partition prune the function sleepEachRow() will be executed twice, - # so 2 seconds for sleepEachRow() and 3 seconds just to ensure that it enters the reading stage. - sleep $((2+3)) + # So let's wait while the reading will be started. + while ! grep -F -q -e "Exception" -e "MergeTreeRangeReader" "$log_file"; do + sleep 0.1 + done # NOTE: parts that are used in query will have refcount increased for each range $CLICKHOUSE_CLIENT -q "select table, name, refcount from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and refcount > 1" @@ -53,6 +53,8 @@ function check_refcnt_for_table() # Kill the query gracefully. kill -INT $PID wait $PID + grep -F Exception "$log_file" | grep -v -F QUERY_WAS_CANCELLED + rm -f "${log_file:?}" } # NOTE: index_granularity=1 to cancel ASAP From a74fa022797678a34de30c891be74caeb5e2802a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 May 2024 14:52:04 +0200 Subject: [PATCH 69/85] tests: reduce pobability of extra parts holders in 02340_parts_refcnt_mergetree Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 5fae571e217..d7e1a562ff4 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -13,6 +13,8 @@ function check_refcnt_for_table() system stop merges $table; -- cleanup thread may hold the parts lock system stop cleanup $table; + -- queue may hold the parts lock for awhile as well + system stop pulling replication log $table; " $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" @@ -64,11 +66,13 @@ $CLICKHOUSE_CLIENT -nmq " create table data_02340 (key Int, part Int) engine=MergeTree() partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340 +$CLICKHOUSE_CLIENT -q "drop table data_02340 sync" $CLICKHOUSE_CLIENT -nmq " drop table if exists data_02340_rep sync; create table data_02340_rep (key Int, part Int) engine=ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340_rep +$CLICKHOUSE_CLIENT -q "drop table data_02340_rep sync" exit 0 From 599470241c7072fd9589102dca635f7c7d063ab7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 May 2024 18:01:02 +0200 Subject: [PATCH 70/85] tests: reduce flakienss of 02340_parts_refcnt_mergetree Signed-off-by: Azat Khuzhin --- .../0_stateless/02340_parts_refcnt_mergetree.reference | 4 ++-- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 8 ++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.reference b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.reference index ae4fafae829..b06fee4af06 100644 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.reference +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.reference @@ -1,2 +1,2 @@ -data_02340 1_2_2_0 6 -data_02340_rep 1_0_0_0 6 +data_02340 1_2_2_0 1 +data_02340_rep 1_0_0_0 1 diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index d7e1a562ff4..caa600298ce 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -49,8 +49,12 @@ function check_refcnt_for_table() sleep 0.1 done - # NOTE: parts that are used in query will have refcount increased for each range - $CLICKHOUSE_CLIENT -q "select table, name, refcount from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and refcount > 1" + # NOTE: parts that are used in query will be holded in multiple places, and + # this is where magic 6 came from. Also there could be some other + # background threads (i.e. asynchronous metrics) that uses the part, so we + # simply filter parts not by "refcount > 1" but with some delta - "3", to + # avoid flakiness. + $CLICKHOUSE_CLIENT -q "select table, name, refcount>=6 from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and refcount >= 3" # Kill the query gracefully. kill -INT $PID From d6fadfc9a6d1ec3e064dc55ae96a13463bb49676 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 May 2024 18:18:18 +0200 Subject: [PATCH 71/85] Azure can do native copy between containers --- src/Backups/BackupIO_AzureBlobStorage.cpp | 8 ++++---- src/Disks/DiskType.cpp | 12 ++++++++++-- .../AzureBlobStorage/AzureObjectStorage.h | 3 ++- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 2 +- src/Storages/StorageAzureBlob.cpp | 8 +------- src/Storages/StorageAzureBlob.h | 2 -- src/TableFunctions/TableFunctionAzureBlobStorage.cpp | 4 ++-- .../TableFunctionAzureBlobStorageCluster.cpp | 4 ++-- .../test_azure_blob_storage_native_copy/test.py | 12 ++++++++---- 9 files changed, 30 insertions(+), 25 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index b035125a545..331cace67d7 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -36,7 +36,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURLWithContainer(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURL().toString(), false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); @@ -47,7 +47,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( std::move(client_ptr), StorageAzureBlob::createSettings(context_), configuration.container, - configuration.getConnectionURLWithContainer()); + configuration.getConnectionURL().toString()); client = object_storage->getAzureBlobStorageClient(); auto settings_copy = *object_storage->getSettings(); @@ -128,7 +128,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const ContextPtr & context_, bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURLWithContainer(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURL().toString(), false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); @@ -138,7 +138,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( std::move(client_ptr), StorageAzureBlob::createSettings(context_), configuration_.container, - configuration.getConnectionURLWithContainer()); + configuration_.getConnectionURL().toString()); client = object_storage->getAzureBlobStorageClient(); auto settings_copy = *object_storage->getSettings(); settings_copy.use_native_copy = allow_azure_native_copy; diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index 448e173a30f..07a7099419b 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -32,8 +32,16 @@ bool DataSourceDescription::operator==(const DataSourceDescription & other) cons bool DataSourceDescription::sameKind(const DataSourceDescription & other) const { - return std::tie(type, object_storage_type, description) - == std::tie(other.type, other.object_storage_type, other.description); + std::string_view our_description = description; + if (our_description.ends_with('/') && our_description.length() > 1) + our_description = our_description.substr(0, our_description.length() - 1); + + std::string_view other_description = other.description; + if (other_description.ends_with('/') && other_description.length() > 1) + other_description = other_description.substr(0, other_description.length() - 1); + + return std::tie(type, object_storage_type, our_description) + == std::tie(other.type, other.object_storage_type, other_description); } std::string DataSourceDescription::toString() const diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 3d94090bcc6..2619c1a1e83 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -173,7 +173,8 @@ private: MultiVersion client; MultiVersion settings; const String object_namespace; /// container + prefix - const String description; /// url + container + + const String description; /// source url without container and prefix LoggerPtr log; }; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index cddcea979b5..cc53054c775 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -313,7 +313,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix, - endpoint.prefix.empty() ? endpoint_string : endpoint_string.substr(0, endpoint_string.length() - endpoint.prefix.length())); + endpoint.getEndpointWithoutContainer()); }; factory.registerObjectStorageType("azure_blob_storage", creator); factory.registerObjectStorageType("azure", creator); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 0103fc0d2a2..e1c6ec0097c 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -303,7 +303,7 @@ void registerStorageAzureBlob(StorageFactory & factory) return std::make_shared( configuration, - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), + std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), args.getContext(), args.table_id, args.columns, @@ -491,12 +491,6 @@ Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); } -std::string StorageAzureBlob::Configuration::getConnectionURLWithContainer() const -{ - auto url = getConnectionURL(); - return fs::path(url.toString()) / container; -} - bool StorageAzureBlob::Configuration::withGlobsIgnorePartitionWildcard() const { if (!withPartitionWildcard()) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 7bce40bce26..b433cd92d68 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -45,8 +45,6 @@ public: Poco::URI getConnectionURL() const; - std::string getConnectionURLWithContainer() const; - std::string connection_url; bool is_connection_string; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index e73277b4d7b..7a17db2a1a8 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -333,7 +333,7 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()); + auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()); if (configuration.format == "auto") return StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, std::nullopt, context).first; return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); @@ -365,7 +365,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct StoragePtr storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), context, StorageID(getDatabaseName(), table_name), columns, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index dc65426a6e3..02b24dccf86 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -39,7 +39,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( /// On worker node this filename won't contains globs storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), context, StorageID(getDatabaseName(), table_name), columns, @@ -54,7 +54,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( storage = std::make_shared( cluster_name, configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURLWithContainer()), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, diff --git a/tests/integration/test_azure_blob_storage_native_copy/test.py b/tests/integration/test_azure_blob_storage_native_copy/test.py index b16d9b4b5c4..a47688d0713 100644 --- a/tests/integration/test_azure_blob_storage_native_copy/test.py +++ b/tests/integration/test_azure_blob_storage_native_copy/test.py @@ -33,7 +33,7 @@ def generate_config(port): local object_storage azure_blob_storage - http://azurite1:{port}/devstoreaccount1 + http://azurite1:{port}/devstoreaccount1/ cont false devstoreaccount1 @@ -45,7 +45,7 @@ def generate_config(port): object_storage azure_blob_storage true - http://azurite1:{port}/devstoreaccount1 + http://azurite1:{port}/devstoreaccount1/ othercontainer false devstoreaccount1 @@ -175,10 +175,14 @@ def test_backup_restore_on_merge_tree_same_container(cluster): assert ( azure_query(node1, f"SELECT * from test_simple_merge_tree_restored") == "1\ta\n" ) + + assert node1.contains_in_log("using native copy") + azure_query(node1, f"DROP TABLE test_simple_merge_tree") azure_query(node1, f"DROP TABLE test_simple_merge_tree_restored") + def test_backup_restore_on_merge_tree_different_container(cluster): node2 = cluster.instances["node2"] azure_query( @@ -196,7 +200,7 @@ def test_backup_restore_on_merge_tree_different_container(cluster): f"BACKUP TABLE test_simple_merge_tree_different_bucket TO {backup_destination}", ) - assert not node2.contains_in_log("using native copy") + assert node2.contains_in_log("using native copy") azure_query( node2, @@ -209,7 +213,7 @@ def test_backup_restore_on_merge_tree_different_container(cluster): == "1\ta\n" ) - assert not node2.contains_in_log("using native copy") + assert node2.contains_in_log("using native copy") azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket") azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket_restored") From 235b0f2d5b2f3a3fc424be7aa5349e379a27dbd1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 20 May 2024 16:28:46 +0000 Subject: [PATCH 72/85] Automatic style fix --- tests/integration/test_azure_blob_storage_native_copy/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_azure_blob_storage_native_copy/test.py b/tests/integration/test_azure_blob_storage_native_copy/test.py index a47688d0713..4f543e4c8b2 100644 --- a/tests/integration/test_azure_blob_storage_native_copy/test.py +++ b/tests/integration/test_azure_blob_storage_native_copy/test.py @@ -182,7 +182,6 @@ def test_backup_restore_on_merge_tree_same_container(cluster): azure_query(node1, f"DROP TABLE test_simple_merge_tree_restored") - def test_backup_restore_on_merge_tree_different_container(cluster): node2 = cluster.instances["node2"] azure_query( From dcb2ed8b9b16db16841dcd52513a2eebd8ad3223 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 May 2024 18:34:47 +0200 Subject: [PATCH 73/85] Update src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 2619c1a1e83..f12ebb68dbb 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -174,7 +174,8 @@ private: MultiVersion settings; const String object_namespace; /// container + prefix - const String description; /// source url without container and prefix + /// We use source url without container and prefix as description, because in Azure there are no limitations for operations between different containers. + const String description; LoggerPtr log; }; From ede7b1474900d0dc7e71fce6dd5e3e7ed1781a00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 May 2024 19:43:39 +0200 Subject: [PATCH 74/85] More of that --- CMakeLists.txt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index afea22a1b90..2d51c1b242f 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -63,7 +63,9 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # Sanitizers are too heavy. Some architectures too. if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64 OR ARCH_LOONGARCH64) - set (RLIMIT_DATA 10000000000) # 10G + # Twice as large + set (RLIMIT_DATA 10000000000) + set (RLIMIT_AS 20000000000) endif() # For some files currently building RISCV64 might be too slow. TODO: Improve compilation times per file From a735ab7dd1f46e4e6fbcd529efafaada5b70afa5 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 20 May 2024 18:58:56 +0000 Subject: [PATCH 75/85] fix upstream commit status update in sync pr --- .github/workflows/pull_request.yml | 2 +- tests/ci/commit_status_helper.py | 36 +++++++++++++----------------- 2 files changed, 16 insertions(+), 22 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 21c2e48677d..f20e987db97 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -178,7 +178,7 @@ jobs: # FinishCheck: if: ${{ !failure() && !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3, CheckReadyForMerge] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 34e2d9f8085..bbda97b9084 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -530,47 +530,41 @@ def update_upstream_sync_status( sync_pr = sync_repo.get_pull(sync_pr_number) # Find the commit that is in both repos, upstream and cloud sync_commits = sync_pr.get_commits().reversed - upstream_commits = upstream_pr.get_commits() + upstream_commits = upstream_pr.get_commits().reversed # Github objects are compared by _url attribute. We can't compare them directly and # should compare commits by SHA1 - upstream_shas = [uc.sha for uc in upstream_commits] + upstream_shas = [c.sha for c in upstream_commits] logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) - sync_shas = [uc.sha for uc in upstream_commits] + sync_shas = [c.sha for c in sync_commits] logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) - found = False - for commit in sync_commits: - try: - idx = upstream_shas.index(commit.sha) - found = True - upstream_commit = upstream_commits[idx] - break - except ValueError: - continue - if not found: - logging.info( - "There's no same commits in upstream and sync PRs, probably force-push" - ) - return + # find latest synced commit + last_synced_upstream_commit = None + for commit in upstream_commits: + if commit.sha in sync_shas: + last_synced_upstream_commit = commit + break + + assert last_synced_upstream_commit sync_status = get_status(state) logging.info( "Using commit %s to post the %s status `%s`: [%s]", - upstream_commit.sha, + last_synced_upstream_commit.sha, sync_status, StatusNames.SYNC, "", ) post_commit_status( - upstream_commit, + last_synced_upstream_commit, sync_status, "", # let's won't expose any urls from cloud "", StatusNames.SYNC, ) trigger_mergeable_check( - upstream_commit, - get_commit_filtered_statuses(upstream_commit), + last_synced_upstream_commit, + get_commit_filtered_statuses(last_synced_upstream_commit), True, set_if_green=can_set_green_mergeable_status, ) From 6c6cdaccd13e32d51fcb41100ca82d6c7ce0c37d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 11 May 2024 00:51:52 +0000 Subject: [PATCH 76/85] Fix filter pushdown for Parquet and maybe StorageMerge --- .../QueryPlan/ReadFromMergeTree.cpp | 20 +------------------ .../QueryPlan/ReadFromPreparedSource.cpp | 1 - .../QueryPlan/SourceStepWithFilter.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 3 ++- src/Storages/IStorageCluster.cpp | 3 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 3 ++- src/Storages/S3Queue/StorageS3Queue.cpp | 3 ++- src/Storages/SelectQueryInfo.cpp | 20 +++++++++++++++++++ src/Storages/SelectQueryInfo.h | 6 ++++++ src/Storages/StorageAzureBlob.cpp | 3 ++- src/Storages/StorageFile.cpp | 3 ++- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageMergeTreeIndex.cpp | 3 ++- src/Storages/StorageS3.cpp | 3 ++- src/Storages/StorageURL.cpp | 3 ++- .../System/IStorageSystemOneBlock.cpp | 3 ++- src/Storages/System/StorageSystemColumns.cpp | 3 ++- .../StorageSystemDataSkippingIndices.cpp | 3 ++- .../System/StorageSystemDetachedParts.cpp | 3 ++- .../System/StorageSystemPartsBase.cpp | 3 ++- src/Storages/System/StorageSystemReplicas.cpp | 3 ++- src/Storages/System/StorageSystemTables.cpp | 3 ++- .../System/StorageSystemZooKeeper.cpp | 3 ++- .../02841_parquet_filter_pushdown.sql | 4 ---- 24 files changed, 63 insertions(+), 43 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e523a2c243c..6f0fa55c349 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1534,25 +1534,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { if (!indexes) { - /// Analyzer generates unique ColumnIdentifiers like __table1.__partition_id in filter nodes, - /// while key analysis still requires unqualified column names. - std::unordered_map node_name_to_input_node_column; - if (query_info.planner_context) - { - const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); - const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); - for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) - { - /// ALIAS columns cannot be used in the filter expression without being calculated in ActionsDAG, - /// so they should not be added to the input nodes. - if (alias_column_expressions.contains(column_name)) - continue; - const auto & column = table_expression_data.getColumnOrThrow(column_name); - node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); - } - } - - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, node_name_to_input_node_column); + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); /// NOTE: Currently we store two DAGs for analysis: /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index b845101125b..92c936cdc20 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,6 +1,5 @@ #include #include -#include #include namespace DB diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index ce5a59a92f9..ad0940b90b9 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -80,7 +80,7 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo void SourceStepWithFilter::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); } void SourceStepWithFilter::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 0f3b03f0955..33bde34b4f9 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -994,7 +994,8 @@ private: void ReadFromHDFS::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index ab45ce877c2..9c5b29ae265 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -86,7 +86,8 @@ private: void ReadFromCluster::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 01417b8977b..1a9aa6d0f41 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -578,7 +578,8 @@ void ReadFromEmbeddedRocksDB::initializePipeline(QueryPipelineBuilder & pipeline void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const auto & sample_block = getOutputStream().header; auto primary_key_data_type = sample_block.getByName(storage.primary_key).type; std::tie(keys, all_scan) = getFilterKeys(storage.primary_key, primary_key_data_type, filter_actions_dag, context); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index c3a772e532c..16e42e32b8a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -287,7 +287,8 @@ void ReadFromS3Queue::createIterator(const ActionsDAG::Node * predicate) void ReadFromS3Queue::applyFilters(ActionDAGNodes added_filter_nodes) { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp index 665da7fee70..d59ccf0dfaf 100644 --- a/src/Storages/SelectQueryInfo.cpp +++ b/src/Storages/SelectQueryInfo.cpp @@ -13,4 +13,24 @@ bool SelectQueryInfo::isFinal() const return select.final(); } +std::unordered_map SelectQueryInfo::buildNodeNameToInputNodeColumn() const +{ + std::unordered_map node_name_to_input_node_column; + if (planner_context) + { + const auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) + { + /// ALIAS columns cannot be used in the filter expression without being calculated in ActionsDAG, + /// so they should not be added to the input nodes. + if (alias_column_expressions.contains(column_name)) + continue; + const auto & column = table_expression_data.getColumnOrThrow(column_name); + node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); + } + } + return node_name_to_input_node_column; +} + } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 655676812d9..11e2a2fc5e7 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -239,5 +239,11 @@ struct SelectQueryInfo bool merge_tree_enable_remove_parts_from_snapshot_optimization = true; bool isFinal() const; + + /// Analyzer generates unique ColumnIdentifiers like __table1.__partition_id in filter nodes, + /// while key analysis still requires unqualified column names. + /// This function generates a map that maps the unique names to table column names, + /// for the current table (`table_expression`). + std::unordered_map buildNodeNameToInputNodeColumn() const; }; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index e1c6ec0097c..e98eaf1e8f2 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -799,7 +799,8 @@ private: void ReadFromAzureBlob::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7db8fc2500a..51bcc64bceb 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1534,7 +1534,8 @@ private: void ReadFromFile::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7afa480149f..f244ca0be05 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1622,7 +1622,7 @@ void ReadFromMerge::applyFilters(const QueryPlan & plan, const ActionDAGNodes & void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); filterTablesAndCreateChildrenPlans(); diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index 4747232d7f7..0b1ad02f8c9 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -280,7 +280,8 @@ private: void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9768653f3fe..2ce188c203c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1454,7 +1454,8 @@ void StorageS3::read( void ReadFromStorageS3Step::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 8a71a771367..272f771194d 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1038,7 +1038,8 @@ private: void ReadFromURL::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/System/IStorageSystemOneBlock.cpp b/src/Storages/System/IStorageSystemOneBlock.cpp index 53399654c8d..456b7c4f90b 100644 --- a/src/Storages/System/IStorageSystemOneBlock.cpp +++ b/src/Storages/System/IStorageSystemOneBlock.cpp @@ -91,7 +91,8 @@ void ReadFromSystemOneBlock::initializePipeline(QueryPipelineBuilder & pipeline, void ReadFromSystemOneBlock::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 74b44cc0a2d..49da1eba9ec 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -342,7 +342,8 @@ private: void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); } diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 2afc03d0e5e..093adc59cc6 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -219,7 +219,8 @@ private: void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 31d566ef8b6..f48a8c67971 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -313,7 +313,8 @@ protected: void ReadFromSystemDetachedParts::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) { const auto * predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index b1ea2dd3f2b..175c0834bcb 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -263,7 +263,8 @@ ReadFromSystemPartsBase::ReadFromSystemPartsBase( void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) { const auto * predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 10d5c353c43..3bd5fd290db 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -290,7 +290,8 @@ private: void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index d428d6bd6d0..1e6c7a0c9ca 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -750,7 +750,8 @@ void StorageSystemTables::read( void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 7afa1894a64..eccddbd5d88 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -474,7 +474,8 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont void ReadFromSystemZooKeeper::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + paths = extractPath(added_filter_nodes.nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); } diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql b/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql index 8521ada04d5..950485d53f0 100644 --- a/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql @@ -8,10 +8,6 @@ set optimize_or_like_chain = 0; set max_block_size = 100000; set max_insert_threads = 1; --- Analyzer breaks the queries with IN and some queries with BETWEEN. --- TODO: Figure out why. -set allow_experimental_analyzer=0; - -- Try all the types. insert into function file('02841.parquet') -- Use negative numbers to test sign extension for signed types and lack of sign extension for From 4ed944285de824d55458aec635bdd45671dbda39 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 11 May 2024 01:19:09 +0000 Subject: [PATCH 77/85] Enable ORC test too --- tests/queries/0_stateless/02892_orc_filter_pushdown.sql | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql index d319252f592..f9aa7696ac6 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -13,9 +13,6 @@ set max_insert_threads = 1; SET session_timezone = 'UTC'; --- Analyzer breaks the queries with IN and some queries with BETWEEN. -set allow_experimental_analyzer=0; - -- Try all the types. insert into function file('02892.orc') From 2e83043d7b0220f08e8bd03d46710d22fe072d57 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 21 May 2024 00:34:54 +0000 Subject: [PATCH 78/85] fix --- src/Storages/StorageMerge.cpp | 2 +- src/Storages/System/StorageSystemZooKeeper.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f244ca0be05..4c678a1228b 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1622,7 +1622,7 @@ void ReadFromMerge::applyFilters(const QueryPlan & plan, const ActionDAGNodes & void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + SourceStepWithFilter::applyFilters(added_filter_nodes); filterTablesAndCreateChildrenPlans(); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index eccddbd5d88..cb46cd19517 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -474,7 +474,7 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont void ReadFromSystemZooKeeper::applyFilters(ActionDAGNodes added_filter_nodes) { - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + SourceStepWithFilter::applyFilters(added_filter_nodes); paths = extractPath(added_filter_nodes.nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); } From a66eab2da14bce638ce8ef6abb51ae5e0125c49d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 21 May 2024 10:13:47 +0200 Subject: [PATCH 79/85] fix clang tidy --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index cc53054c775..c83b9247b99 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -306,7 +306,6 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) bool /* skip_access_check */) -> ObjectStoragePtr { AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix); - std::string endpoint_string = endpoint.getEndpoint(); return createObjectStorage( ObjectStorageType::Azure, config, config_prefix, name, From 0b3102576e6137d2783068fa7f325c96f57a37c2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 May 2024 11:43:16 +0200 Subject: [PATCH 80/85] One attempt to fix native copy for azure --- .../copyAzureBlobStorageFile.cpp | 21 +++++++--- .../test.py | 39 ++++++++++++++++++- 2 files changed, 54 insertions(+), 6 deletions(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 667e63729ca..38eb97e17f4 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -300,21 +300,32 @@ void copyAzureBlobStorageFile( if (size < settings->max_single_part_copy_size) { + LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy blob sync {} -> {}", src_blob, dest_blob); block_blob_client_dest.CopyFromUri(source_uri); } else { Azure::Storage::Blobs::StartBlobCopyOperation operation = block_blob_client_dest.StartCopyFromUri(source_uri); - // Wait for the operation to finish, checking for status every 100 second. auto copy_response = operation.PollUntilDone(std::chrono::milliseconds(100)); auto properties_model = copy_response.Value; - if (properties_model.CopySource.HasValue()) - { - throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy failed"); - } + auto copy_status = properties_model.CopyStatus; + auto copy_status_description = properties_model.CopyStatusDescription; + + if (copy_status.HasValue() && copy_status.Value() == Azure::Storage::Blobs::Models::CopyStatus::Success) + { + LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy of {} to {} finished", properties_model.CopySource.Value(), dest_blob); + } + else + { + if (copy_status.HasValue()) + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} failed with status {} description {} (operation is done {})", + src_blob, dest_blob, copy_status.Value().ToString(), copy_status_description.Value(), operation.IsDone()); + else + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} didn't complete with sucess status (operation is done {})", src_blob, dest_blob, operation.IsDone()); + } } } else diff --git a/tests/integration/test_azure_blob_storage_native_copy/test.py b/tests/integration/test_azure_blob_storage_native_copy/test.py index a47688d0713..3169696a4d7 100644 --- a/tests/integration/test_azure_blob_storage_native_copy/test.py +++ b/tests/integration/test_azure_blob_storage_native_copy/test.py @@ -110,6 +110,11 @@ def cluster(): main_configs=[path], with_azurite=True, ) + cluster.add_instance( + "node3", + main_configs=[path], + with_azurite=True, + ) cluster.start() yield cluster @@ -182,7 +187,6 @@ def test_backup_restore_on_merge_tree_same_container(cluster): azure_query(node1, f"DROP TABLE test_simple_merge_tree_restored") - def test_backup_restore_on_merge_tree_different_container(cluster): node2 = cluster.instances["node2"] azure_query( @@ -217,3 +221,36 @@ def test_backup_restore_on_merge_tree_different_container(cluster): azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket") azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket_restored") + + +def test_backup_restore_on_merge_tree_native_copy_async(cluster): + node3 = cluster.instances["node3"] + azure_query( + node3, + f"CREATE TABLE test_simple_merge_tree_async(key UInt64, data String) Engine = MergeTree() ORDER BY tuple() SETTINGS storage_policy='policy_azure_cache'", + ) + azure_query(node3, f"INSERT INTO test_simple_merge_tree_async VALUES (1, 'a')") + + backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_merge_tree_async_backup')" + print("BACKUP DEST", backup_destination) + azure_query( + node3, + f"BACKUP TABLE test_simple_merge_tree_async TO {backup_destination}", + settings={"azure_max_single_part_copy_size": 0} + ) + + assert node3.contains_in_log("using native copy") + + azure_query( + node3, + f"RESTORE TABLE test_simple_merge_tree_async AS test_simple_merge_tree_async_restored FROM {backup_destination};", + settings={"azure_max_single_part_copy_size": 0} + ) + assert ( + azure_query(node3, f"SELECT * from test_simple_merge_tree_async_restored") == "1\ta\n" + ) + + assert node3.contains_in_log("using native copy") + + azure_query(node3, f"DROP TABLE test_simple_merge_tree_async") + azure_query(node3, f"DROP TABLE test_simple_merge_tree_async_restored") From a6dd13814cfcc14d8f56d1c7ec142f109d01c369 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 21 May 2024 09:53:22 +0000 Subject: [PATCH 81/85] Automatic style fix --- .../test_azure_blob_storage_native_copy/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_azure_blob_storage_native_copy/test.py b/tests/integration/test_azure_blob_storage_native_copy/test.py index 3169696a4d7..77d400240b1 100644 --- a/tests/integration/test_azure_blob_storage_native_copy/test.py +++ b/tests/integration/test_azure_blob_storage_native_copy/test.py @@ -236,7 +236,7 @@ def test_backup_restore_on_merge_tree_native_copy_async(cluster): azure_query( node3, f"BACKUP TABLE test_simple_merge_tree_async TO {backup_destination}", - settings={"azure_max_single_part_copy_size": 0} + settings={"azure_max_single_part_copy_size": 0}, ) assert node3.contains_in_log("using native copy") @@ -244,10 +244,11 @@ def test_backup_restore_on_merge_tree_native_copy_async(cluster): azure_query( node3, f"RESTORE TABLE test_simple_merge_tree_async AS test_simple_merge_tree_async_restored FROM {backup_destination};", - settings={"azure_max_single_part_copy_size": 0} + settings={"azure_max_single_part_copy_size": 0}, ) assert ( - azure_query(node3, f"SELECT * from test_simple_merge_tree_async_restored") == "1\ta\n" + azure_query(node3, f"SELECT * from test_simple_merge_tree_async_restored") + == "1\ta\n" ) assert node3.contains_in_log("using native copy") From 86c77b998ddc332a476d348639f05f7b63ee51e5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 May 2024 12:49:57 +0200 Subject: [PATCH 82/85] Fix global query profiler Signed-off-by: Azat Khuzhin --- src/Interpreters/ThreadStatusExt.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6607df8d9af..9ca521a4ab3 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -462,8 +462,8 @@ void ThreadStatus::initGlobalProfiler([[maybe_unused]] UInt64 global_profiler_re { #if !defined(SANITIZER) && !defined(__APPLE__) /// profilers are useless without trace collector - auto global_context_ptr = global_context.lock(); - if (!global_context_ptr || !global_context_ptr->hasTraceCollector()) + auto context = Context::getGlobalContextInstance(); + if (!context->hasTraceCollector()) return; try From ef182b035679a4f88968441b9c9ebeb6b6195b06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 May 2024 12:54:46 +0200 Subject: [PATCH 83/85] Fix --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 38eb97e17f4..207b3cde868 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -324,7 +324,7 @@ void copyAzureBlobStorageFile( throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} failed with status {} description {} (operation is done {})", src_blob, dest_blob, copy_status.Value().ToString(), copy_status_description.Value(), operation.IsDone()); else - throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} didn't complete with sucess status (operation is done {})", src_blob, dest_blob, operation.IsDone()); + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} didn't complete with success status (operation is done {})", src_blob, dest_blob, operation.IsDone()); } } } From c10bafc094eb4b2913e191d2781d574ae05966b6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 20:39:35 +0000 Subject: [PATCH 84/85] Incorporate feedback --- .../functions/other-functions.md | 110 +++++++++++++----- 1 file changed, 84 insertions(+), 26 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7c930d23919..95591a7f483 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1000,7 +1000,8 @@ If executed in the context of a distributed table, this function generates a nor ## blockNumber -Returns the sequence number of the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks) where the row is located. +Returns a monotonically increasing sequence number of the [block](../../development/architecture.md#block) containing the row. +The returned block number is updated on a best-effort basis, i.e. it may not be fully accurate **Syntax** @@ -1017,26 +1018,43 @@ blockNumber() Query: ```sql -SELECT groupUniqArray(blockNumber()) -FROM -( +SELECT blockNumber() +FROM ( SELECT * - FROM system.numbers_mt - LIMIT 100000 -) + FROM system.numbers + LIMIT 10 +) SETTINGS max_block_size = 2 ``` Result: ```response -┌─groupUniqArray(blockNumber())─┐ -│ [6,7] │ -└───────────────────────────────┘ +┌─blockNumber()─┐ +│ 7 │ +│ 7 │ +└───────────────┘ +┌─blockNumber()─┐ +│ 8 │ +│ 8 │ +└───────────────┘ +┌─blockNumber()─┐ +│ 9 │ +│ 9 │ +└───────────────┘ +┌─blockNumber()─┐ +│ 10 │ +│ 10 │ +└───────────────┘ +┌─blockNumber()─┐ +│ 11 │ +│ 11 │ +└───────────────┘ ``` ## rowNumberInBlock {#rowNumberInBlock} -Returns the ordinal number of the row in the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks). Different data blocks are always recalculated. +Returns for each [block](../../development/architecture.md#block) processed by `rowNumberInBlock` the number of the current row. +The returned number starts for each block at 0. **Syntax** @@ -1053,23 +1071,42 @@ rowNumberInBlock() Query: ```sql -SELECT - min(rowNumberInBlock()), - max(rowNumberInBlock()) -FROM (SELECT * FROM system.numbers_mt LIMIT 100000); +SELECT rowNumberInBlock() +FROM ( + SELECT * + FROM system.numbers_mt + LIMIT 10 +) SETTINGS max_block_size = 2 ``` Result: ```response -┌─min(rowNumberInBlock())─┬─max(rowNumberInBlock())─┐ -│ 0 │ 65408 │ -└─────────────────────────┴─────────────────────────┘ +┌─rowNumberInBlock()─┐ +│ 0 │ +│ 1 │ +└────────────────────┘ +┌─rowNumberInBlock()─┐ +│ 0 │ +│ 1 │ +└────────────────────┘ +┌─rowNumberInBlock()─┐ +│ 0 │ +│ 1 │ +└────────────────────┘ +┌─rowNumberInBlock()─┐ +│ 0 │ +│ 1 │ +└────────────────────┘ +┌─rowNumberInBlock()─┐ +│ 0 │ +│ 1 │ +└────────────────────┘ ``` ## rowNumberInAllBlocks -Returns the ordinal number of the row in the [data block](https://clickhouse.com/docs/en/integrations/python#data-blocks). This function only considers the affected data blocks. +Returns a unique row number for each row processed by `rowNumberInAllBlocks`. The returned numbers start at 0. **Syntax** @@ -1086,18 +1123,39 @@ rowNumberInAllBlocks() Query: ```sql -SELECT - min(rowNumberInAllBlocks()), - max(rowNumberInAllBlocks()) -FROM (SELECT * FROM system.numbers_mt LIMIT 100000); +SELECT rowNumberInAllBlocks() +FROM +( + SELECT * + FROM system.numbers_mt + LIMIT 10 +) +SETTINGS max_block_size = 2 ``` Result: ```response -┌─min(rowNumberInAllBlocks())─┬─max(rowNumberInAllBlocks())─┐ -│ 0 │ 99999 │ -└─────────────────────────────┴─────────────────────────────┘ +┌─rowNumberInAllBlocks()─┐ +│ 0 │ +│ 1 │ +└────────────────────────┘ +┌─rowNumberInAllBlocks()─┐ +│ 4 │ +│ 5 │ +└────────────────────────┘ +┌─rowNumberInAllBlocks()─┐ +│ 2 │ +│ 3 │ +└────────────────────────┘ +┌─rowNumberInAllBlocks()─┐ +│ 6 │ +│ 7 │ +└────────────────────────┘ +┌─rowNumberInAllBlocks()─┐ +│ 8 │ +│ 9 │ +└────────────────────────┘ ``` From cd742dd040747f70ea92aedfded53565beea1253 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 20:41:20 +0000 Subject: [PATCH 85/85] Fix whitespace errors --- docs/en/sql-reference/functions/other-functions.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 95591a7f483..98cb5631553 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1001,7 +1001,7 @@ If executed in the context of a distributed table, this function generates a nor ## blockNumber Returns a monotonically increasing sequence number of the [block](../../development/architecture.md#block) containing the row. -The returned block number is updated on a best-effort basis, i.e. it may not be fully accurate +The returned block number is updated on a best-effort basis, i.e. it may not be fully accurate. **Syntax** @@ -1019,7 +1019,8 @@ Query: ```sql SELECT blockNumber() -FROM ( +FROM +( SELECT * FROM system.numbers LIMIT 10 @@ -1072,7 +1073,8 @@ Query: ```sql SELECT rowNumberInBlock() -FROM ( +FROM +( SELECT * FROM system.numbers_mt LIMIT 10