From 852318449f2ab8e0bfaeca9904b90936d751d177 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 20:17:43 +0200 Subject: [PATCH 001/496] Check perf tests --- src/Common/PODArray_fwd.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PODArray_fwd.h b/src/Common/PODArray_fwd.h index d570a90e467..bd780eb51b0 100644 --- a/src/Common/PODArray_fwd.h +++ b/src/Common/PODArray_fwd.h @@ -17,12 +17,12 @@ constexpr size_t integerRoundUp(size_t value, size_t dividend) } template , size_t pad_right_ = 0, + typename TAllocator = Allocator, size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray; /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ -template > +template > using PaddedPODArray = PODArray; /** A helper for declaring PODArray that uses inline memory. @@ -32,6 +32,6 @@ using PaddedPODArray = PODArray using PODArrayWithStackMemory = PODArray, rounded_bytes, alignof(T)>>; + AllocatorWithStackMemory, rounded_bytes, alignof(T)>>; } From 7568de2202a7fa99539d8d46092315bf3c7fe5e6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 22:03:53 +0100 Subject: [PATCH 002/496] Revert "Check perf tests" This reverts commit 852318449f2ab8e0bfaeca9904b90936d751d177. --- src/Common/PODArray_fwd.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PODArray_fwd.h b/src/Common/PODArray_fwd.h index bd780eb51b0..d570a90e467 100644 --- a/src/Common/PODArray_fwd.h +++ b/src/Common/PODArray_fwd.h @@ -17,12 +17,12 @@ constexpr size_t integerRoundUp(size_t value, size_t dividend) } template , size_t pad_right_ = 0, + typename TAllocator = Allocator, size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray; /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ -template > +template > using PaddedPODArray = PODArray; /** A helper for declaring PODArray that uses inline memory. @@ -32,6 +32,6 @@ using PaddedPODArray = PODArray using PODArrayWithStackMemory = PODArray, rounded_bytes, alignof(T)>>; + AllocatorWithStackMemory, rounded_bytes, alignof(T)>>; } From 57d036e5899b52fd4fdab9447630e01bf3d5382b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 22:04:24 +0100 Subject: [PATCH 003/496] impl --- tests/performance/scripts/perf.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index e98c158249a..94f145d82db 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -427,6 +427,8 @@ for query_index in queries_to_run: for conn_index, c in enumerate(this_query_connections): try: + c.execute("SYSTEM JEMALLOC PURGE") + res = c.execute( q, query_id=run_id, From e348186ba26072eb76e549cd2a0adcd801c92bc6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 18 Jun 2024 21:30:37 +0000 Subject: [PATCH 004/496] clear hint in table engine and sources --- src/Access/Common/AccessRightsElement.cpp | 6 +- src/Access/Common/AccessRightsElement.h | 1 + src/Access/ContextAccess.cpp | 80 ++++++++++++++++++----- 3 files changed, 70 insertions(+), 17 deletions(-) diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 24ff4e7631b..2ee13d6b94f 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -224,7 +224,11 @@ void AccessRightsElement::replaceEmptyDatabase(const String & current_database) String AccessRightsElement::toString() const { return toStringImpl(*this, true); } String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); } - +String AccessRightsElement::toStringWithoutONClause() const +{ + String result{access_flags.toKeywords().front()}; + return result + " ON {db.table}"; +} bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index ba625fc43df..49764fc727f 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -89,6 +89,7 @@ struct AccessRightsElement /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; + String toStringWithoutONClause() const; }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 2a658d7aaa2..3ce30a0b681 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -37,6 +37,24 @@ namespace ErrorCodes namespace { + static const std::vector> source_and_table_engines = { + {AccessType::FILE, "File"}, + {AccessType::URL, "URL"}, + {AccessType::REMOTE, "Distributed"}, + {AccessType::MONGO, "MongoDB"}, + {AccessType::REDIS, "Redis"}, + {AccessType::MYSQL, "MySQL"}, + {AccessType::POSTGRES, "PostgreSQL"}, + {AccessType::SQLITE, "SQLite"}, + {AccessType::ODBC, "ODBC"}, + {AccessType::JDBC, "JDBC"}, + {AccessType::HDFS, "HDFS"}, + {AccessType::S3, "S3"}, + {AccessType::HIVE, "Hive"}, + {AccessType::AZURE, "AzureBlobStorage"} + }; + + AccessRights mixAccessRightsFromUserAndRoles(const User & user, const EnabledRolesInfo & roles_info) { AccessRights res = user.access; @@ -205,22 +223,6 @@ namespace } /// There is overlap between AccessType sources and table engines, so the following code avoids user granting twice. - static const std::vector> source_and_table_engines = { - {AccessType::FILE, "File"}, - {AccessType::URL, "URL"}, - {AccessType::REMOTE, "Distributed"}, - {AccessType::MONGO, "MongoDB"}, - {AccessType::REDIS, "Redis"}, - {AccessType::MYSQL, "MySQL"}, - {AccessType::POSTGRES, "PostgreSQL"}, - {AccessType::SQLITE, "SQLite"}, - {AccessType::ODBC, "ODBC"}, - {AccessType::JDBC, "JDBC"}, - {AccessType::HDFS, "HDFS"}, - {AccessType::S3, "S3"}, - {AccessType::HIVE, "Hive"}, - {AccessType::AZURE, "AzureBlobStorage"} - }; /// Sync SOURCE and TABLE_ENGINE, so only need to check TABLE_ENGINE later. if (access_control.doesTableEnginesRequireGrant()) @@ -555,6 +557,18 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() return nothing_granted; } +/// Just Dummy to pass compile. +template +static std::string_view getTableEngineName(const Args &... args[[maybe_unused]]) +{ + return ""; +} + +template +static std::string_view getTableEngineName(std::string_view name, const Args &... args[[maybe_unused]]) +{ + return name; +} template bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const @@ -611,6 +625,40 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading since + /// SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. + if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) + { + AccessFlags newFlags; + + String table_engine_name{getTableEngineName(args...)}; + for (const auto & source_and_table_engine : source_and_table_engines) + { + const auto & table_engine = std::get<1>(source_and_table_engine); + if (table_engine != table_engine_name) continue; + const auto & source = std::get<0>(source_and_table_engine); + /// Set the flags from Table Engine to SOURCES so that prompts can be meaningful. + newFlags = source; + break; + } + + if (newFlags == AccessType::NONE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); + + if (grant_option && acs->isGranted(flags, args...)) + { + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. " + "The required privileges have been granted, but without grant option. " + "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", + AccessRightsElement{newFlags}.toStringWithoutONClause()); + } + + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", + AccessRightsElement{newFlags}.toStringWithoutONClause() + (grant_option ? " WITH GRANT OPTION" : "")); + } + if (grant_option && acs->isGranted(flags, args...)) { return access_denied(ErrorCodes::ACCESS_DENIED, From f7e81e1ae2752020c076990395349ccd2d69cf2b Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 18 Jun 2024 21:59:07 +0000 Subject: [PATCH 005/496] fix --- src/Access/ContextAccess.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 3ce30a0b681..de0e7e3d777 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -625,8 +625,8 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { - /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading since - /// SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading + /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) { AccessFlags newFlags; From a2ee0668f12c8cd1b88b8c4ad46c15271a5a1fd2 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 19 Jun 2024 02:20:22 +0000 Subject: [PATCH 006/496] fix --- src/Access/ContextAccess.cpp | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index de0e7e3d777..4620561053b 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -37,7 +37,7 @@ namespace ErrorCodes namespace { - static const std::vector> source_and_table_engines = { + const std::vector> source_and_table_engines = { {AccessType::FILE, "File"}, {AccessType::URL, "URL"}, {AccessType::REMOTE, "Distributed"}, @@ -268,6 +268,11 @@ namespace template std::string_view getDatabase(std::string_view arg1, const OtherArgs &...) { return arg1; } + + std::string_view getTableEngine() { return {}; } + + template + std::string_view getTableEngine(std::string_view arg1, const OtherArgs &...) { return arg1; } } @@ -557,18 +562,6 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() return nothing_granted; } -/// Just Dummy to pass compile. -template -static std::string_view getTableEngineName(const Args &... args[[maybe_unused]]) -{ - return ""; -} - -template -static std::string_view getTableEngineName(std::string_view name, const Args &... args[[maybe_unused]]) -{ - return name; -} template bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const @@ -631,7 +624,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg { AccessFlags newFlags; - String table_engine_name{getTableEngineName(args...)}; + String table_engine_name{getTableEngine(args...)}; for (const auto & source_and_table_engine : source_and_table_engines) { const auto & table_engine = std::get<1>(source_and_table_engine); @@ -642,7 +635,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg break; } - if (newFlags == AccessType::NONE) + if (newFlags.isEmpty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); if (grant_option && acs->isGranted(flags, args...)) From b125f8166f32648572c2ed0d540ded56a97ac628 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 19 Jun 2024 16:41:50 +0100 Subject: [PATCH 007/496] impl --- tests/performance/array_reduce.xml | 15 ++++++--------- tests/performance/scripts/perf.py | 14 ++++++++++++-- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/performance/array_reduce.xml b/tests/performance/array_reduce.xml index 8e271c155f3..0f50eba43bf 100644 --- a/tests/performance/array_reduce.xml +++ b/tests/performance/array_reduce.xml @@ -1,11 +1,8 @@ - - - - SELECT arrayReduce('count', range(100000000)) - SELECT arrayReduce('sum', range(100000000)) - SELECT arrayReduceInRanges('count', [(1, 100000000)], range(100000000)) - SELECT arrayReduceInRanges('sum', [(1, 100000000)], range(100000000)) - SELECT arrayReduceInRanges('count', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] - SELECT arrayReduceInRanges('sum', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] + SELECT arrayReduce('count', range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduce('sum', range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('count', [(1, 1000000)], range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('sum', [(1, 1000000)], range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('count', arrayZip(range(1000000), range(1000000)), range(1000000))[123456] + SELECT arrayReduceInRanges('sum', arrayZip(range(1000000), range(1000000)), range(1000000))[123456] diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index 94f145d82db..f89784a0e0b 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -345,6 +345,18 @@ for query_index in queries_to_run: print(f"display-name\t{query_index}\t{tsv_escape(query_display_name)}") + for conn_index, c in enumerate(all_connections): + try: + c.execute("SYSTEM JEMALLOC PURGE") + + print( + f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}" + ) + except KeyboardInterrupt: + raise + except: + continue + # Prewarm: run once on both servers. Helps to bring the data into memory, # precompile the queries, etc. # A query might not run on the old server if it uses a function added in the @@ -427,8 +439,6 @@ for query_index in queries_to_run: for conn_index, c in enumerate(this_query_connections): try: - c.execute("SYSTEM JEMALLOC PURGE") - res = c.execute( q, query_id=run_id, From 6f1f416700a32cf95af15b79543e27cbcffe2f14 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 17:37:05 +0100 Subject: [PATCH 008/496] one more test --- tests/performance/final_big_column.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 1fd586d2d90..4bfdfdf804f 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -1,6 +1,7 @@ - 1 + + 8 20G @@ -10,8 +11,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers_mt(5000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers_mt(5000000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null From 91dc9a69d844d781c7d4f94ca01d0a9bbe1a1f29 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 21 Jun 2024 22:03:38 +0100 Subject: [PATCH 009/496] fix final_big_column --- tests/performance/final_big_column.xml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 4bfdfdf804f..bc7c3570db6 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -1,7 +1,6 @@ - - 8 + 1 20G @@ -11,8 +10,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers_mt(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers_mt(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(1000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(1000000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null From cb0a692ba061d910712f0144cb2b5308db5d033f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 21 Jun 2024 23:42:56 +0100 Subject: [PATCH 010/496] fix read_from_comp_parts --- tests/performance/read_from_comp_parts.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/performance/read_from_comp_parts.xml b/tests/performance/read_from_comp_parts.xml index a625075588a..055df51d12d 100644 --- a/tests/performance/read_from_comp_parts.xml +++ b/tests/performance/read_from_comp_parts.xml @@ -5,15 +5,16 @@ ORDER BY (c1, c2) SETTINGS min_rows_for_wide_part = 1000000000 AS SELECT * - FROM generateRandom('c1 UInt32, c2 UInt64, s1 String, arr1 Array(UInt32), c3 UInt64, s2 String', 0, 30, 30) + FROM generateRandom('c1 UInt32, c2 UInt64, s1 String, arr1 Array(UInt32), c3 UInt64, s2 String', 0, 5, 6) LIMIT 50000000 + SETTINGS max_insert_threads = 8 8 - SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1) + SELECT count() FROM mt_comp_parts WHERE NOT ignore(s1) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c2, s1, arr1, s2) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1, s1, c3) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1, c2, c3) From 051290e6c912108986c896916db087c71230a121 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 24 Jun 2024 12:26:46 +0000 Subject: [PATCH 011/496] Add throughput introspection for scheduler queues --- src/Common/EventRateMeter.h | 18 ++++++++++++++++-- src/Common/Scheduler/ISchedulerNode.h | 14 ++++++++++++++ src/Common/Scheduler/Nodes/FairPolicy.h | 3 +-- src/Common/Scheduler/Nodes/FifoQueue.h | 3 +-- src/Common/Scheduler/Nodes/PriorityPolicy.h | 3 +-- .../Scheduler/Nodes/SemaphoreConstraint.h | 3 +-- .../Scheduler/Nodes/ThrottlerConstraint.h | 3 +-- src/Common/Scheduler/SchedulerRoot.h | 3 +-- src/Storages/System/StorageSystemScheduler.cpp | 2 ++ 9 files changed, 38 insertions(+), 14 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index 3a21a80ce8b..4c38d1d9371 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -14,8 +14,9 @@ namespace DB class EventRateMeter { public: - explicit EventRateMeter(double now, double period_) + explicit EventRateMeter(double now, double period_, double step_ = 0.0) : period(period_) + , step(step_) , half_decay_time(period * std::numbers::ln2) // for `ExponentiallySmoothedAverage::sumWeights()` to be equal to `1/period` { reset(now); @@ -38,7 +39,16 @@ public: if (now - period <= start) // precise counting mode events = ExponentiallySmoothedAverage(events.value + count, now); else // exponential smoothing mode - events.add(count, now, half_decay_time); + { + // Adding events too often lead to low precision due to smoothing too often, so we buffer new events and add them in steps + step_count += count; + if (step_start + step <= now) + { + events.add(step_count, now, half_decay_time); + step_start = now; + step_count = 0; + } + } } /// Compute average event rate throughout `[now - period, now]` period. @@ -58,16 +68,20 @@ public: void reset(double now) { start = now; + step_start = now; events = ExponentiallySmoothedAverage(); data_points = 0; } private: const double period; + const double step; // duration of a step const double half_decay_time; double start; // Instant in past without events before it; when measurement started or reset ExponentiallySmoothedAverage events; // Estimated number of events in the last `period` size_t data_points = 0; + double step_start; // start instant of the last step + double step_count = 0.0; // number of events accumulated since step start }; } diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 81b491b0eda..c051829e336 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -176,6 +178,14 @@ protected: /// Postponed to be handled in scheduler thread, so it is intended to be called from outside. void scheduleActivation(); + /// Helper for introspection metrics + void incrementDequeued(ResourceCost cost) + { + dequeued_requests++; + dequeued_cost += cost; + throughput.add(static_cast(clock_gettime_ns())/1e9, cost); + } + public: EventQueue * const event_queue; String basename; @@ -189,6 +199,10 @@ public: std::atomic dequeued_cost{0}; std::atomic canceled_cost{0}; std::atomic busy_periods{0}; + + /// Average dequeued_cost per second + /// WARNING: Should only be accessed from the scheduler thread, so that locking is not required + EventRateMeter throughput{static_cast(clock_gettime_ns())/1e9, 2, 1}; }; using SchedulerNodePtr = std::shared_ptr; diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index 0a4e55c253b..fba637e979e 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -188,8 +188,7 @@ public: if (request) { - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, heap_size > 0}; } } diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 9ec997c06d2..9fbc6d1ae65 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -59,8 +59,7 @@ public: if (requests.empty()) busy_periods++; queue_cost -= result->cost; - dequeued_requests++; - dequeued_cost += result->cost; + incrementDequeued(result->cost); return {result, !requests.empty()}; } diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 22a5155cfeb..91dc95600d5 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -122,8 +122,7 @@ public: if (request) { - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, !items.empty()}; } } diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 10fce536f5d..92c6af9db18 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -81,8 +81,7 @@ public: child_active = child_now_active; if (!active()) busy_periods++; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, active()}; } diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index f4a5795bb2b..56866336f50 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -89,8 +89,7 @@ public: child_active = child_now_active; if (!active()) busy_periods++; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, active()}; } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 7af42fdbbea..5307aadc3cc 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -162,8 +162,7 @@ public: if (request == nullptr) // Possible in case of request cancel, just retry continue; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, current != nullptr}; } } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 339a59e88a5..b42c807d6fc 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -31,6 +31,7 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() {"dequeued_requests", std::make_shared(), "The total number of resource requests dequeued from this node."}, {"canceled_requests", std::make_shared(), "The total number of resource requests canceled from this node."}, {"dequeued_cost", std::make_shared(), "The sum of costs (e.g. size in bytes) of all requests dequeued from this node."}, + {"throughput", std::make_shared(), "Current average throughput (dequeued cost per second)."}, {"canceled_cost", std::make_shared(), "The sum of costs (e.g. size in bytes) of all requests canceled from this node."}, {"busy_periods", std::make_shared(), "The total number of deactivations of this node."}, {"vruntime", std::make_shared(std::make_shared()), @@ -96,6 +97,7 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c res_columns[i++]->insert(node->dequeued_requests.load()); res_columns[i++]->insert(node->canceled_requests.load()); res_columns[i++]->insert(node->dequeued_cost.load()); + res_columns[i++]->insert(node->throughput.rate(static_cast(clock_gettime_ns())/1e9)); res_columns[i++]->insert(node->canceled_cost.load()); res_columns[i++]->insert(node->busy_periods.load()); From b0ac0327d4d38a918e9be8fc499038cc43cafb2c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 25 Jun 2024 11:48:29 +0000 Subject: [PATCH 012/496] Fix bug in EventRateMeter It was relying on ExponentiallySmoothedCounter::get() which is designed for specific 1 second time interval between points. Now sum of weights is computed separatly in `duration` field, giving very accurate measurements independent of interval. --- src/Common/EventRateMeter.h | 52 ++++++---------- src/Common/ProgressIndication.h | 2 +- src/Common/tests/gtest_event_rate_meter.cpp | 68 +++++++++++++++++++++ 3 files changed, 86 insertions(+), 36 deletions(-) create mode 100644 src/Common/tests/gtest_event_rate_meter.cpp diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index 4c38d1d9371..b8a9112428f 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -4,8 +4,6 @@ #include -#include - namespace DB { @@ -14,10 +12,10 @@ namespace DB class EventRateMeter { public: - explicit EventRateMeter(double now, double period_, double step_ = 0.0) + explicit EventRateMeter(double now, double period_, size_t heating_ = 0) : period(period_) - , step(step_) - , half_decay_time(period * std::numbers::ln2) // for `ExponentiallySmoothedAverage::sumWeights()` to be equal to `1/period` + , max_interval(period * 10) + , heating(heating_) { reset(now); } @@ -30,25 +28,11 @@ public: { // Remove data for initial heating stage that can present at the beginning of a query. // Otherwise it leads to wrong gradual increase of average value, turning algorithm into not very reactive. - if (count != 0.0 && ++data_points < 5) - { - start = events.time; - events = ExponentiallySmoothedAverage(); - } + if (count != 0.0 && data_points++ <= heating) + reset(events.time, data_points); - if (now - period <= start) // precise counting mode - events = ExponentiallySmoothedAverage(events.value + count, now); - else // exponential smoothing mode - { - // Adding events too often lead to low precision due to smoothing too often, so we buffer new events and add them in steps - step_count += count; - if (step_start + step <= now) - { - events.add(step_count, now, half_decay_time); - step_start = now; - step_count = 0; - } - } + duration.add(std::min(max_interval, now - duration.time), now, period); + events.add(count, now, period); } /// Compute average event rate throughout `[now - period, now]` period. @@ -59,29 +43,27 @@ public: add(now, 0); if (unlikely(now <= start)) return 0; - if (now - period <= start) // precise counting mode - return events.value / (now - start); - else // exponential smoothing mode - return events.get(half_decay_time); // equals to `events.value / period` + + // We do not use .get() because sum of weights will anyway be canceled out (optimization) + return events.value / duration.value; } - void reset(double now) + void reset(double now, size_t data_points_ = 0) { start = now; - step_start = now; events = ExponentiallySmoothedAverage(); - data_points = 0; + duration = ExponentiallySmoothedAverage(); + data_points = data_points_; } private: const double period; - const double step; // duration of a step - const double half_decay_time; + const double max_interval; + const size_t heating; double start; // Instant in past without events before it; when measurement started or reset - ExponentiallySmoothedAverage events; // Estimated number of events in the last `period` + ExponentiallySmoothedAverage duration; // Current duration of a period + ExponentiallySmoothedAverage events; // Estimated number of events in last `duration` seconds size_t data_points = 0; - double step_start; // start instant of the last step - double step_count = 0.0; // number of events accumulated since step start }; } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index a9965785889..d925077a072 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -91,7 +91,7 @@ private: bool write_progress_on_update = false; - EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 2'000'000'000 /*ns*/}; // average cpu utilization last 2 second + EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 2'000'000'000 /*ns*/, 4}; // average cpu utilization last 2 second, skip first 4 points HostToTimesMap hosts_data; /// In case of all of the above: /// - clickhouse-local diff --git a/src/Common/tests/gtest_event_rate_meter.cpp b/src/Common/tests/gtest_event_rate_meter.cpp new file mode 100644 index 00000000000..91ceec5eef7 --- /dev/null +++ b/src/Common/tests/gtest_event_rate_meter.cpp @@ -0,0 +1,68 @@ +#include + +#include + +#include + + +TEST(EventRateMeter, ExponentiallySmoothedAverage) +{ + double target = 100.0; + + // The test is only correct for timestep of 1 second because of + // how sum of weights is implemented inside `ExponentiallySmoothedAverage` + double time_step = 1.0; + + for (double half_decay_time : { 0.1, 1.0, 10.0, 100.0}) + { + DB::ExponentiallySmoothedAverage esa; + + int steps = static_cast(half_decay_time * 30 / time_step); + for (int i = 1; i <= steps; ++i) + esa.add(target * time_step, i * time_step, half_decay_time); + double measured = esa.get(half_decay_time); + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } +} + +TEST(EventRateMeter, ConstantRate) +{ + double target = 100.0; + + for (double period : {0.1, 1.0, 10.0}) + { + for (double time_step : {0.001, 0.01, 0.1, 1.0}) + { + DB::EventRateMeter erm(0.0, period); + + int steps = static_cast(period * 30 / time_step); + for (int i = 1; i <= steps; ++i) + erm.add(i * time_step, target * time_step); + double measured = erm.rate(steps * time_step); + // std::cout << "T=" << period << " dt=" << time_step << " measured=" << measured << std::endl; + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } + } +} + +TEST(EventRateMeter, PreciseStart) +{ + double target = 100.0; + + for (double period : {0.1, 1.0, 10.0}) + { + for (double time_step : {0.001, 0.01, 0.1, 1.0}) + { + DB::EventRateMeter erm(0.0, period); + + int steps = static_cast(period / time_step); + for (int i = 1; i <= steps; ++i) + { + erm.add(i * time_step, target * time_step); + double measured = erm.rate(i * time_step); + // std::cout << "T=" << period << " dt=" << time_step << " measured=" << measured << std::endl; + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } + } + } +} From c998ec1e4f1b91f8ca20c2bd5a7acb6ac8d2e1b1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 29 Jun 2024 02:40:22 +0000 Subject: [PATCH 013/496] add test and better naming --- src/Access/Common/AccessRightsElement.cpp | 4 +- src/Access/Common/AccessRightsElement.h | 2 +- src/Access/ContextAccess.cpp | 10 +-- ...xml => config_with_table_engine_grant.xml} | 0 .../config_without_table_engine_grant.xml | 5 ++ ...est.py => test_with_table_engine_grant.py} | 2 +- .../test_without_table_engine_grant.py | 81 +++++++++++++++++++ 7 files changed, 95 insertions(+), 9 deletions(-) rename tests/integration/test_grant_and_revoke/configs/{config.xml => config_with_table_engine_grant.xml} (100%) create mode 100644 tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml rename tests/integration/test_grant_and_revoke/{test.py => test_with_table_engine_grant.py} (99%) create mode 100644 tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 2ee13d6b94f..63bda09a51b 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -224,10 +224,10 @@ void AccessRightsElement::replaceEmptyDatabase(const String & current_database) String AccessRightsElement::toString() const { return toStringImpl(*this, true); } String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); } -String AccessRightsElement::toStringWithoutONClause() const +String AccessRightsElement::toStringForAccessTypeSource() const { String result{access_flags.toKeywords().front()}; - return result + " ON {db.table}"; + return result + " ON *.*"; } bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 49764fc727f..78e94e6f2e4 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -89,7 +89,7 @@ struct AccessRightsElement /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; - String toStringWithoutONClause() const; + String toStringForAccessTypeSource() const; }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 4620561053b..8ff1fc8ed21 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -622,7 +622,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) { - AccessFlags newFlags; + AccessFlags new_flags; String table_engine_name{getTableEngine(args...)}; for (const auto & source_and_table_engine : source_and_table_engines) @@ -631,11 +631,11 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (table_engine != table_engine_name) continue; const auto & source = std::get<0>(source_and_table_engine); /// Set the flags from Table Engine to SOURCES so that prompts can be meaningful. - newFlags = source; + new_flags = source; break; } - if (newFlags.isEmpty()) + if (new_flags.isEmpty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); if (grant_option && acs->isGranted(flags, args...)) @@ -644,12 +644,12 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg "{}: Not enough privileges. " "The required privileges have been granted, but without grant option. " "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", - AccessRightsElement{newFlags}.toStringWithoutONClause()); + AccessRightsElement{new_flags}.toStringForAccessTypeSource()); } return access_denied(ErrorCodes::ACCESS_DENIED, "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", - AccessRightsElement{newFlags}.toStringWithoutONClause() + (grant_option ? " WITH GRANT OPTION" : "")); + AccessRightsElement{new_flags}.toStringForAccessTypeSource() + (grant_option ? " WITH GRANT OPTION" : "")); } if (grant_option && acs->isGranted(flags, args...)) diff --git a/tests/integration/test_grant_and_revoke/configs/config.xml b/tests/integration/test_grant_and_revoke/configs/config_with_table_engine_grant.xml similarity index 100% rename from tests/integration/test_grant_and_revoke/configs/config.xml rename to tests/integration/test_grant_and_revoke/configs/config_with_table_engine_grant.xml diff --git a/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml b/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml new file mode 100644 index 00000000000..d3571f281f5 --- /dev/null +++ b/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml @@ -0,0 +1,5 @@ + + + false + + diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py similarity index 99% rename from tests/integration/test_grant_and_revoke/test.py rename to tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index e533cced1e4..25ca7913e4e 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -5,7 +5,7 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", - main_configs=["configs/config.xml"], + main_configs=["configs/config_with_table_engine_grant.xml"], user_configs=["configs/users.d/users.xml"], ) diff --git a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py new file mode 100644 index 00000000000..210bb8ec465 --- /dev/null +++ b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py @@ -0,0 +1,81 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=["configs/config_without_table_engine_grant.xml"], + user_configs=["configs/users.d/users.xml"], +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + + instance.query("CREATE DATABASE test") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + instance.query("DROP USER IF EXISTS A") + instance.query("DROP TABLE IF EXISTS test.table1") + + +def test_table_engine_and_source_grant(): + instance.query("DROP USER IF EXISTS A") + instance.query("CREATE USER A") + instance.query("GRANT CREATE TABLE ON test.table1 TO A") + + instance.query("GRANT POSTGRES ON *.* TO A") + + instance.query( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("DROP TABLE test.table1") + + instance.query("REVOKE POSTGRES ON *.* FROM A") + + assert "Not enough privileges" in instance.query_and_get_error( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + # expecting grant POSTGRES instead of grant PostgreSQL due to discrepancy between source access type and table engine + assert "grant POSTGRES ON *.*" in instance.query_and_get_error( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("GRANT SOURCES ON *.* TO A") + + instance.query( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("DROP TABLE test.table1") From 57818990f201562d0b6938c1b8de78d16bac471f Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 2 Jul 2024 20:41:08 +0000 Subject: [PATCH 014/496] fix the test --- src/Access/ContextAccess.cpp | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 8ff1fc8ed21..f534c334318 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -618,6 +618,22 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { + auto access_denied_no_grant = [&](AccessFlags access_flags, FmtArgs && ...fmt_args) + { + if (grant_option && acs->isGranted(access_flags, fmt_args...)) + { + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. " + "The required privileges have been granted, but without grant option. " + "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", + AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions()); + } + + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", + AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : "")); + }; + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) @@ -635,8 +651,9 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg break; } + /// Might happen in the case of grant Table Engine on A (but not source), then revoke A. if (new_flags.isEmpty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); + return access_denied_no_grant(flags, args...); if (grant_option && acs->isGranted(flags, args...)) { @@ -652,18 +669,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg AccessRightsElement{new_flags}.toStringForAccessTypeSource() + (grant_option ? " WITH GRANT OPTION" : "")); } - if (grant_option && acs->isGranted(flags, args...)) - { - return access_denied(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. " - "The required privileges have been granted, but without grant option. " - "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", - AccessRightsElement{flags, args...}.toStringWithoutOptions()); - } - - return access_denied(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", - AccessRightsElement{flags, args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : "")); + return access_denied_no_grant(flags, args...); } struct PrecalculatedFlags From 5d16ba57aa84ef82ccf7e34a4635ad1d14e7859d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Jul 2024 14:46:49 +0000 Subject: [PATCH 015/496] Update version_date.tsv and changelogs after v24.3.5.46-lts --- docs/changelogs/v24.3.5.46-lts.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 41 insertions(+) create mode 100644 docs/changelogs/v24.3.5.46-lts.md diff --git a/docs/changelogs/v24.3.5.46-lts.md b/docs/changelogs/v24.3.5.46-lts.md new file mode 100644 index 00000000000..1f2b7c8b0b7 --- /dev/null +++ b/docs/changelogs/v24.3.5.46-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.5.46-lts (fe54cead6b6) FIXME as compared to v24.3.4.147-lts (31a7bdc346d) + +#### Improvement +* Backported in [#65463](https://github.com/ClickHouse/ClickHouse/issues/65463): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Backported in [#65882](https://github.com/ClickHouse/ClickHouse/issues/65882): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65302](https://github.com/ClickHouse/ClickHouse/issues/65302): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65892](https://github.com/ClickHouse/ClickHouse/issues/65892): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65283](https://github.com/ClickHouse/ClickHouse/issues/65283): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65370](https://github.com/ClickHouse/ClickHouse/issues/65370): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#65446](https://github.com/ClickHouse/ClickHouse/issues/65446): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65708](https://github.com/ClickHouse/ClickHouse/issues/65708): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65352](https://github.com/ClickHouse/ClickHouse/issues/65352): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65327](https://github.com/ClickHouse/ClickHouse/issues/65327): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#65538](https://github.com/ClickHouse/ClickHouse/issues/65538): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65576](https://github.com/ClickHouse/ClickHouse/issues/65576): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65159](https://github.com/ClickHouse/ClickHouse/issues/65159): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65615](https://github.com/ClickHouse/ClickHouse/issues/65615): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65728](https://github.com/ClickHouse/ClickHouse/issues/65728): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65261](https://github.com/ClickHouse/ClickHouse/issues/65261): Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#65667](https://github.com/ClickHouse/ClickHouse/issues/65667): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65784](https://github.com/ClickHouse/ClickHouse/issues/65784): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65929](https://github.com/ClickHouse/ClickHouse/issues/65929): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65824](https://github.com/ClickHouse/ClickHouse/issues/65824): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65223](https://github.com/ClickHouse/ClickHouse/issues/65223): Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#65901](https://github.com/ClickHouse/ClickHouse/issues/65901): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8112ed9083b..8e748a2c2ca 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.5.1.1763-stable 2024-06-01 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 From 849fb83c9770fedb937dc59df73c0cc172e115bf Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Jul 2024 17:37:17 +0800 Subject: [PATCH 016/496] add function printf --- .../functions/string-replace-functions.md | 25 ++ src/Functions/printf.cpp | 308 ++++++++++++++++++ .../0_stateless/032010_printf.reference | 16 + tests/queries/0_stateless/032010_printf.sql | 24 ++ 4 files changed, 373 insertions(+) create mode 100644 src/Functions/printf.cpp create mode 100644 tests/queries/0_stateless/032010_printf.reference create mode 100644 tests/queries/0_stateless/032010_printf.sql diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 8793ebdd1a3..177790c983e 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -223,3 +223,28 @@ SELECT translateUTF8('Münchener Straße', 'üß', 'us') AS res; │ Munchener Strase │ └──────────────────┘ ``` + +## printf + +The `printf` function formats the given string with the values (strings, integers, floating-points etc.) listed in the arguments, similar to printf function in C++. The format string can contain format specifiers starting with `%` character. Anything not contained in `%` and the following format specifier is considered literal text and copied verbatim into the output. Literal `%` character can be escaped by `%%`. + +**Syntax** + +``` sql +printf(format, arg1, arg2, ...) +``` + +**Example** + +Query: + +``` sql +select printf('%%%s %s %d', 'Hello', 'World', 2024); +``` + + +``` response +┌─printf('%%%s %s %d', 'Hello', 'World', 2024)─┐ +│ %Hello World 2024 │ +└──────────────────────────────────────────────┘ +``` diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp new file mode 100644 index 00000000000..cb21d5e39ad --- /dev/null +++ b/src/Functions/printf.cpp @@ -0,0 +1,308 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +class FunctionPrintf : public IFunction +{ +private: + ContextPtr context; + FunctionOverloadResolverPtr function_concat; + + struct Instruction + { + std::string_view format; + size_t rows; + bool is_literal; /// format is literal string without any argument + ColumnWithTypeAndName input; /// Only used when is_literal is false + + ColumnWithTypeAndName execute() + { + if (is_literal) + return executeLiteral(format); + else if (isColumnConst(*input.column)) + return executeConstant(input); + else + return executeNonconstant(input); + } + + String toString() const + { + std::ostringstream oss; + oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() + << std::endl; + return oss.str(); + } + + private: + ColumnWithTypeAndName executeLiteral(std::string_view literal) + { + ColumnWithTypeAndName res; + auto str_col = ColumnString::create(); + str_col->insert(fmt::sprintf(literal)); + res.column = ColumnConst::create(std::move(str_col), rows); + res.type = std::make_shared(); + return res; + } + + ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) + { + ColumnWithTypeAndName tmp_arg = arg; + const auto & const_col = static_cast(*arg.column); + tmp_arg.column = const_col.getDataColumnPtr(); + + ColumnWithTypeAndName tmp_res = executeNonconstant(tmp_arg); + return ColumnWithTypeAndName{ColumnConst::create(tmp_res.column, arg.column->size()), tmp_res.type, tmp_res.name}; + } + + ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) + { + size_t size = arg.column->size(); + auto res_col = ColumnString::create(); + auto & res_str = static_cast(*res_col); + auto & res_offsets = res_str.getOffsets(); + auto & res_chars = res_str.getChars(); + res_offsets.reserve_exact(size); + res_chars.reserve(format.size() * size * 2); + + String s; + WhichDataType which(arg.type); + +#define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ + else if (which.IS_TYPE()) \ + { \ + for (size_t i = 0; i < size; ++i) \ + { \ + auto a = arg.column->GET_TYPE(i); \ + s = fmt::sprintf(format, a); \ + res_str.insertData(s.data(), s.size()); \ + } \ + } + + if (false) + ; + EXECUTE_BY_TYPE(isNativeInt, getInt) + EXECUTE_BY_TYPE(isNativeUInt, getUInt) + EXECUTE_BY_TYPE(isFloat32, getFloat32) + EXECUTE_BY_TYPE(isFloat64, getFloat64) + else if (which.isStringOrFixedString()) + { + for (size_t i = 0; i < size; ++i) + { + auto a = arg.column->getDataAt(i).toView(); + s = fmt::sprintf(format, a); + res_str.insertData(s.data(), s.size()); + } + } + else throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument type of function {} is {}, but native numeric or string type is expected", + FunctionPrintf::name, + arg.type->getName()); +#undef EXECUTE_BY_TYPE + + ColumnWithTypeAndName res; + res.name = arg.name; + res.type = std::make_shared(); + res.column = std::move(res_col); + return res; + } + }; + +public: + static constexpr auto name = "printf"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + explicit FunctionPrintf(ContextPtr context_) + : context(context_), function_concat(FunctionFactory::instance().get("concat", context)) { } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool useDefaultImplementationForConstants() const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at least 1", + getName(), + arguments.size()); + + /// First pattern argument must have string type + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The first argument type of function {} is {}, but String type is expected", + getName(), + arguments[0]->getName()); + + for (size_t i = 1; i < arguments.size(); ++i) + { + if (!isNativeNumber(arguments[i]) && !isStringOrFixedString(arguments[i])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The {}-th argument type of function {} is {}, but native numeric or string type is expected", + i + 1, + getName(), + arguments[i]->getName()); + } + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const ColumnPtr & c0 = arguments[0].column; + const ColumnConst * c0_const_string = typeid_cast(&*c0); + if (!c0_const_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function {} must be constant string", getName()); + + String format = c0_const_string->getValue(); + auto instructions = buildInstructions(format, arguments, input_rows_count); + + ColumnsWithTypeAndName concat_args(instructions.size()); + for (size_t i = 0; i < instructions.size(); ++i) + { + std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + concat_args[i] = instructions[i].execute(); + std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + } + + auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); + return res; + } + +private: + std::vector buildInstructions(const String & format , const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + { + std::vector instructions; + instructions.reserve(arguments.size()); + + auto append_instruction = [&](const char * begin, const char * end, const ColumnWithTypeAndName & arg) + { + Instruction instr; + instr.rows = input_rows_count; + instr.format = std::string_view(begin, end - begin); + + size_t size = end - begin; + if (size > 1 && begin[0] == '%' and begin[1] != '%') + { + instr.is_literal = false; + instr.input = arg; + } + else + { + instr.is_literal = true; + } + instructions.emplace_back(std::move(instr)); + }; + + auto check_index_range = [&](size_t idx) + { + if (idx >= arguments.size()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, but format is {}", + getName(), + arguments.size(), + format); + }; + + const char * begin = format.data(); + const char * end = format.data() + format.size(); + const char * curr = begin; + size_t idx = 0; + while (curr < end) + { + const char * tmp = curr; + bool is_first = curr == begin; /// If current instruction is the first one + bool is_literal = false; /// If current instruction is literal string without any argument + if (is_first) + { + if (*curr != '%') + is_literal = true; + else if (curr + 1 < end && *(curr + 1) == '%') + is_literal = true; + else + ++idx; /// Skip first argument if first instruction is not literal + } + + if (!is_literal) + ++curr; + + while (curr < end) + { + if (*curr != '%') + ++curr; + else if (curr + 1 < end && *(curr + 1) == '%') + curr += 2; + else + { + check_index_range(idx); + append_instruction(tmp, curr, arguments[idx]); + ++idx; + break; + } + } + + if (curr == end) + { + check_index_range(idx); + append_instruction(tmp, curr, arguments[idx]); + ++idx; + } + } + + /// Check if all arguments are used + if (idx != arguments.size()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, but format is {}", + getName(), + arguments.size(), + format); + + return instructions; + } +}; + +} + +REGISTER_FUNCTION(Printf) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/032010_printf.reference b/tests/queries/0_stateless/032010_printf.reference new file mode 100644 index 00000000000..58501cbd0fc --- /dev/null +++ b/tests/queries/0_stateless/032010_printf.reference @@ -0,0 +1,16 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/032010_printf.sql b/tests/queries/0_stateless/032010_printf.sql new file mode 100644 index 00000000000..58fe081e499 --- /dev/null +++ b/tests/queries/0_stateless/032010_printf.sql @@ -0,0 +1,24 @@ +-- Testing integer formats +select printf('%%d: %d', 123) = '%d: 123'; +select printf('%%i: %i', 123) = '%i: 123'; +select printf('%%u: %u', 123) = '%u: 123'; +select printf('%%o: %o', 123) = '%o: 173'; +select printf('%%x: %x', 123) = '%x: 7b'; +select printf('%%X: %X', 123) = '%X: 7B'; + +-- Testing floating point formats +select printf('%%f: %f', 123.456) = '%f: 123.456000'; +select printf('%%F: %F', 123.456) = '%F: 123.456000'; +select printf('%%e: %e', 123.456) = '%e: 1.234560e+02'; +select printf('%%E: %E', 123.456) = '%E: 1.234560E+02'; +select printf('%%g: %g', 123.456) = '%g: 123.456'; +select printf('%%G: %G', 123.456) = '%G: 123.456'; +select printf('%%a: %a', 123.456) = '%a: 0x1.edd2f1a9fbe77p+6'; +select printf('%%A: %A', 123.456) = '%A: 0X1.EDD2F1A9FBE77P+6'; + +-- Testing character formats +select printf('%%s: %s', 'abc') = '%s: abc'; + + +-- Testing the %% specifier +select printf('%%%%: %%') = '%%: %'; \ No newline at end of file From e0e3842772ead940f53346cc087ea54e5e6aa8fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Jul 2024 10:15:33 +0800 Subject: [PATCH 017/496] support printf --- src/Functions/printf.cpp | 6 +++--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index cb21d5e39ad..b2a0143a3f8 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -52,7 +52,7 @@ private: String toString() const { - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() << std::endl; return oss.str(); @@ -195,9 +195,9 @@ public: ColumnsWithTypeAndName concat_args(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) { - std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; concat_args[i] = instructions[i].execute(); - std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; } auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 943caf918d6..21a9b759466 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2264,6 +2264,7 @@ prettyspacemonoblock prettyspacenoescapes prettyspacenoescapesmonoblock prewhere +printf privateKeyFile privateKeyPassphraseHandler prlimit From 8b1bc00e9a6462a6dce2946510d83a85bc69a139 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Jul 2024 10:52:53 +0800 Subject: [PATCH 018/496] fix style --- src/Functions/printf.cpp | 33 +++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index b2a0143a3f8..c7c6bd228a7 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -9,19 +9,19 @@ #include #include +#include #include #include #include #include -#include namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int ILLEGAL_COLUMN; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -54,7 +54,7 @@ private: { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() - << std::endl; + << std::endl; return oss.str(); } @@ -93,15 +93,15 @@ private: WhichDataType which(arg.type); #define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ - else if (which.IS_TYPE()) \ - { \ - for (size_t i = 0; i < size; ++i) \ - { \ - auto a = arg.column->GET_TYPE(i); \ - s = fmt::sprintf(format, a); \ - res_str.insertData(s.data(), s.size()); \ - } \ - } + else if (which.IS_TYPE()) \ + { \ + for (size_t i = 0; i < size; ++i) \ + { \ + auto a = arg.column->GET_TYPE(i); \ + s = fmt::sprintf(format, a); \ + res_str.insertData(s.data(), s.size()); \ + } \ + } if (false) ; @@ -205,7 +205,8 @@ public: } private: - std::vector buildInstructions(const String & format , const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + std::vector + buildInstructions(const String & format, const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { std::vector instructions; instructions.reserve(arguments.size()); @@ -248,7 +249,7 @@ private: { const char * tmp = curr; bool is_first = curr == begin; /// If current instruction is the first one - bool is_literal = false; /// If current instruction is literal string without any argument + bool is_literal = false; /// If current instruction is literal string without any argument if (is_first) { if (*curr != '%') From 96d063bcc39712c5a21a8e51244a9e216af8536a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 12:10:44 +0800 Subject: [PATCH 019/496] renmae ut files --- src/Functions/printf.cpp | 118 +++++++++++------- ...erence => 03203_function_printf.reference} | 0 ...0_printf.sql => 03203_function_printf.sql} | 0 3 files changed, 73 insertions(+), 45 deletions(-) rename tests/queries/0_stateless/{032010_printf.reference => 03203_function_printf.reference} (100%) rename tests/queries/0_stateless/{032010_printf.sql => 03203_function_printf.sql} (100%) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index c7c6bd228a7..247c4a65daf 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -40,7 +40,7 @@ private: bool is_literal; /// format is literal string without any argument ColumnWithTypeAndName input; /// Only used when is_literal is false - ColumnWithTypeAndName execute() + ColumnWithTypeAndName execute() const { if (is_literal) return executeLiteral(format); @@ -50,7 +50,7 @@ private: return executeNonconstant(input); } - String toString() const + [[maybe_unused]] String toString() const { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() @@ -59,7 +59,7 @@ private: } private: - ColumnWithTypeAndName executeLiteral(std::string_view literal) + ColumnWithTypeAndName executeLiteral(std::string_view literal) const { ColumnWithTypeAndName res; auto str_col = ColumnString::create(); @@ -69,7 +69,7 @@ private: return res; } - ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) + ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) const { ColumnWithTypeAndName tmp_arg = arg; const auto & const_col = static_cast(*arg.column); @@ -79,57 +79,85 @@ private: return ColumnWithTypeAndName{ColumnConst::create(tmp_res.column, arg.column->size()), tmp_res.type, tmp_res.name}; } - ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) + template + bool executeNumber(const IColumn & column, ColumnString::Chars & res_chars, ColumnString::Offsets & res_offsets) const + { + const ColumnVector * concrete_column = checkAndGetColumn>(&column); + if (!concrete_column) + return false; + + String s; + size_t curr_offset = 0; + const auto & data = concrete_column->getData(); + for (size_t i = 0; i < data.size(); ++i) + { + T a = data[i]; + s = fmt::sprintf(format, static_cast>(a)); + memcpy(&res_chars[curr_offset], s.data(), s.size()); + res_chars[curr_offset + s.size()] = 0; + + curr_offset += s.size() + 1; + res_offsets[i] = curr_offset; + } + return true; + } + + template + bool executeString(const IColumn & column, ColumnString::Chars & res_chars, ColumnString::Offsets & res_offsets) const + { + const COLUMN * concrete_column = checkAndGetColumn(&column); + if (!concrete_column) + return false; + + String s; + size_t curr_offset = 0; + for (size_t i = 0; i < concrete_column->size(); ++i) + { + auto a = concrete_column->getDataAt(i).toView(); + s = fmt::sprintf(format, a); + memcpy(&res_chars[curr_offset], s.data(), s.size()); + res_chars[curr_offset + s.size()] = 0; + + curr_offset += s.size() + 1; + res_offsets[i] = curr_offset; + } + return true; + } + + ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) const { size_t size = arg.column->size(); auto res_col = ColumnString::create(); auto & res_str = static_cast(*res_col); auto & res_offsets = res_str.getOffsets(); auto & res_chars = res_str.getChars(); - res_offsets.reserve_exact(size); - res_chars.reserve(format.size() * size * 2); + res_offsets.resize_exact(size); + res_chars.reserve(format.size() * size); - String s; WhichDataType which(arg.type); - -#define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ - else if (which.IS_TYPE()) \ - { \ - for (size_t i = 0; i < size; ++i) \ - { \ - auto a = arg.column->GET_TYPE(i); \ - s = fmt::sprintf(format, a); \ - res_str.insertData(s.data(), s.size()); \ - } \ - } - - if (false) - ; - EXECUTE_BY_TYPE(isNativeInt, getInt) - EXECUTE_BY_TYPE(isNativeUInt, getUInt) - EXECUTE_BY_TYPE(isFloat32, getFloat32) - EXECUTE_BY_TYPE(isFloat64, getFloat64) - else if (which.isStringOrFixedString()) + if (which.isNativeNumber() + && (executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets))) { - for (size_t i = 0; i < size; ++i) - { - auto a = arg.column->getDataAt(i).toView(); - s = fmt::sprintf(format, a); - res_str.insertData(s.data(), s.size()); - } + return {std::move(res_col), std::make_shared(), arg.name}; } - else throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The argument type of function {} is {}, but native numeric or string type is expected", - FunctionPrintf::name, - arg.type->getName()); -#undef EXECUTE_BY_TYPE - - ColumnWithTypeAndName res; - res.name = arg.name; - res.type = std::make_shared(); - res.column = std::move(res_col); - return res; + else if ( + which.isStringOrFixedString() + && (executeString(*arg.column, res_chars, res_offsets) + || executeString(*arg.column, res_chars, res_offsets))) + { + return {std::move(res_col), std::make_shared(), arg.name}; + } + else + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument type of function {} is {}, but native numeric or string type is expected", + FunctionPrintf::name, + arg.type->getName()); } }; diff --git a/tests/queries/0_stateless/032010_printf.reference b/tests/queries/0_stateless/03203_function_printf.reference similarity index 100% rename from tests/queries/0_stateless/032010_printf.reference rename to tests/queries/0_stateless/03203_function_printf.reference diff --git a/tests/queries/0_stateless/032010_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql similarity index 100% rename from tests/queries/0_stateless/032010_printf.sql rename to tests/queries/0_stateless/03203_function_printf.sql From 88851ddb569f9ae8c61420bde99d2ad5f3d76889 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 12:15:44 +0800 Subject: [PATCH 020/496] improve uts --- .../0_stateless/03203_function_printf.reference | 5 +++++ .../queries/0_stateless/03203_function_printf.sql | 14 ++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_function_printf.reference b/tests/queries/0_stateless/03203_function_printf.reference index 58501cbd0fc..338ecb0183d 100644 --- a/tests/queries/0_stateless/03203_function_printf.reference +++ b/tests/queries/0_stateless/03203_function_printf.reference @@ -14,3 +14,8 @@ 1 1 1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03203_function_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql index 58fe081e499..c41cbf0b5e9 100644 --- a/tests/queries/0_stateless/03203_function_printf.sql +++ b/tests/queries/0_stateless/03203_function_printf.sql @@ -19,6 +19,16 @@ select printf('%%A: %A', 123.456) = '%A: 0X1.EDD2F1A9FBE77P+6'; -- Testing character formats select printf('%%s: %s', 'abc') = '%s: abc'; - -- Testing the %% specifier -select printf('%%%%: %%') = '%%: %'; \ No newline at end of file +select printf('%%%%: %%') = '%%: %'; + +-- Testing integer formats with precision +select printf('%%.5d: %.5d', 123) = '%.5d: 00123'; + +-- Testing floating point formats with precision +select printf('%%.2f: %.2f', 123.456) = '%.2f: 123.46'; +select printf('%%.2e: %.2e', 123.456) = '%.2e: 1.23e+02'; +select printf('%%.2g: %.2g', 123.456) = '%.2g: 1.2e+02'; + +-- Testing character formats with precision +select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; \ No newline at end of file From d988399aa68ad8a6bc412bfc48d9fdefe63c1657 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 14:32:24 +0800 Subject: [PATCH 021/496] fix failed uts --- src/Functions/printf.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index 247c4a65daf..a890b886338 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -141,7 +141,9 @@ private: || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) - || executeNumber(*arg.column, res_chars, res_offsets))) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets))) { return {std::move(res_col), std::make_shared(), arg.name}; } From 21ca5f2d65c936a2c5b5fbc8f3f0c40d0ce60a6a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 15:32:05 +0800 Subject: [PATCH 022/496] fix failed ut --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index a152066a460..873b6bbb660 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -562,6 +562,7 @@ positionCaseInsensitive positionCaseInsensitiveUTF8 positionUTF8 pow +printf proportionsZTest protocol queryID From c6b558c7915b070167649d4e88eafb2613570bd3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 4 Jul 2024 22:30:18 +0200 Subject: [PATCH 023/496] Done --- .../02814_currentDatabase_for_table_functions.sql | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql index 74b5cf5f432..8b1e3ba1e10 100644 --- a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql @@ -13,7 +13,13 @@ CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT J CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); INSERT INTO null_table_buffer VALUES (1); -SELECT sleep(3) FORMAT Null; + +-- OPTIMIZE query should flush Buffer table, but still it is not guaranteed +-- (see the comment StorageBuffer::optimize) +-- But the combination of OPTIMIZE + sleep + OPTIMIZE should be enough. +OPTIMIZE TABLE null_table_buffer; +SELECT sleep(1) FORMAT Null; +OPTIMIZE TABLE null_table_buffer; -- Insert about should've landed into `null_mv` SELECT count() FROM null_mv; From ca7e003c6d7af6bf0676bba7cb61ab560c202bf3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Jul 2024 15:27:03 +0000 Subject: [PATCH 024/496] Fixed test --- .../02814_currentDatabase_for_table_functions.reference | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference index 7ff95106d3d..20b14d9a67b 100644 --- a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference @@ -11,7 +11,12 @@ CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); INSERT INTO null_table_buffer VALUES (1); -SELECT sleep(3) FORMAT Null; +-- OPTIMIZE query should flush Buffer table, but still it is not guaranteed +-- (see the comment StorageBuffer::optimize) +-- But the combination of OPTIMIZE + sleep + OPTIMIZE should be enough. +OPTIMIZE TABLE null_table_buffer; +SELECT sleep(1) FORMAT Null; +OPTIMIZE TABLE null_table_buffer; -- Insert about should've landed into `null_mv` SELECT count() FROM null_mv; 1 From 264d7d760fedd3fc3c900d13ee1f7976887efaa7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 15:52:23 +0000 Subject: [PATCH 025/496] Bump rocksdb to 7.0.4 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 9 +++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index be366233921..4fc59e24001 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit be366233921293bd07a84dc4ea6991858665f202 +Subproject commit 4fc59e240016a62180b09703e2938c3d7e928de0 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 3a14407166c..f6479346063 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -59,10 +59,8 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") endif() -set (HAVE_THREAD_LOCAL 1) -if(HAVE_THREAD_LOCAL) - add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) -endif() +# thread_local is part of C++11 and later (TODO: clean up this define) +add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) @@ -182,7 +180,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/env.cc ${ROCKSDB_SOURCE_DIR}/env/env_chroot.cc ${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc - ${ROCKSDB_SOURCE_DIR}/env/env_hdfs.cc ${ROCKSDB_SOURCE_DIR}/env/file_system.cc ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc @@ -311,7 +308,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc ${ROCKSDB_SOURCE_DIR}/util/random.cc ${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc - ${ROCKSDB_SOURCE_DIR}/util/regex.cc ${ROCKSDB_SOURCE_DIR}/util/ribbon_config.cc ${ROCKSDB_SOURCE_DIR}/util/slice.cc ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc @@ -335,6 +331,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters.cc ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc + ${ROCKSDB_SOURCE_DIR}/utilities/counted_fs.cc ${ROCKSDB_SOURCE_DIR}/utilities/debug.cc ${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc ${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc From 5b6956ea234962cca5414a3fb0a6191407b4305a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 16:48:09 +0000 Subject: [PATCH 026/496] Bump rocksdb to v7.10.2 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 53 +++++++++++++------------- contrib/rocksdb-cmake/build_version.cc | 31 +++++++++++---- 3 files changed, 51 insertions(+), 35 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 4fc59e24001..01e43568fa9 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 4fc59e240016a62180b09703e2938c3d7e928de0 +Subproject commit 01e43568fa9f3f7bf107b2b66c00b286b456f33e diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index f6479346063..98790158baa 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,6 +5,9 @@ if (NOT ENABLE_ROCKSDB) return() endif() +# not in original build system, otherwise xxHash.cc fails to compile with ClickHouse C++23 default +set (CMAKE_CXX_STANDARD 20) + # Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs option(WITH_JEMALLOC "build with JeMalloc" OFF) @@ -16,14 +19,6 @@ option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) -# third-party/folly is only validated to work on Linux and Windows for now. -# So only turn it on there by default. -if(CMAKE_SYSTEM_NAME MATCHES "Linux|Windows") - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON) -else() - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) -endif() - if(WITH_SNAPPY) add_definitions(-DSNAPPY) list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) @@ -44,7 +39,7 @@ if(WITH_ZSTD) list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) endif() -option(PORTABLE "build a portable binary" ON) +add_definitions(-DROCKSDB_PORTABLE) if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) add_definitions(-DHAVE_SSE42) @@ -59,9 +54,6 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") endif() -# thread_local is part of C++11 and later (TODO: clean up this define) -add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) - if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") @@ -87,19 +79,21 @@ set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") include_directories(${ROCKSDB_SOURCE_DIR}) include_directories("${ROCKSDB_SOURCE_DIR}/include") -if(WITH_FOLLY_DISTRIBUTED_MUTEX) - include_directories("${ROCKSDB_SOURCE_DIR}/third-party/folly") -endif() set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/cache.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_key.cc + ${ROCKSDB_SOURCE_DIR}/cache/cache_helpers.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_reservation_manager.cc + ${ROCKSDB_SOURCE_DIR}/cache/charged_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/compressed_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc ${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_contents.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc @@ -111,6 +105,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_source.cc ${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc ${ROCKSDB_SOURCE_DIR}/db/builder.cc ${ROCKSDB_SOURCE_DIR}/db/c.cc @@ -122,7 +117,11 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_fifo.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_level.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_universal.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_service_job.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_state.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_outputs.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/sst_partitioner.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/subcompaction_state.cc ${ROCKSDB_SOURCE_DIR}/db/convenience.cc ${ROCKSDB_SOURCE_DIR}/db/db_filesnapshot.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/compacted_db_impl.cc @@ -157,10 +156,11 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc ${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc ${ROCKSDB_SOURCE_DIR}/db/output_validator.cc - ${ROCKSDB_SOURCE_DIR}/db/periodic_work_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/db/periodic_task_scheduler.cc ${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc ${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc ${ROCKSDB_SOURCE_DIR}/db/repair.cc + ${ROCKSDB_SOURCE_DIR}/db/seqno_to_time_mapping.cc ${ROCKSDB_SOURCE_DIR}/db/snapshot_impl.cc ${ROCKSDB_SOURCE_DIR}/db/table_cache.cc ${ROCKSDB_SOURCE_DIR}/db/table_properties_collector.cc @@ -172,6 +172,8 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/version_set.cc ${ROCKSDB_SOURCE_DIR}/db/wal_edit.cc ${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc + ${ROCKSDB_SOURCE_DIR}/db/wide/wide_column_serialization.cc + ${ROCKSDB_SOURCE_DIR}/db/wide/wide_columns.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc ${ROCKSDB_SOURCE_DIR}/db/write_controller.cc @@ -230,16 +232,17 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/options/options.cc ${ROCKSDB_SOURCE_DIR}/options/options_helper.cc ${ROCKSDB_SOURCE_DIR}/options/options_parser.cc + ${ROCKSDB_SOURCE_DIR}/port/mmap.cc ${ROCKSDB_SOURCE_DIR}/port/stack_trace.cc ${ROCKSDB_SOURCE_DIR}/table/adaptive/adaptive_table_factory.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/binary_search_index_reader.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block.cc - ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_filter_block.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_builder.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_factory.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_iterator.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_reader.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_cache.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefetcher.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefix_index.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_hash_index.cc @@ -297,9 +300,12 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_result.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc + ${ROCKSDB_SOURCE_DIR}/util/async_file_reader.cc + ${ROCKSDB_SOURCE_DIR}/util/cleanable.cc ${ROCKSDB_SOURCE_DIR}/util/coding.cc ${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc ${ROCKSDB_SOURCE_DIR}/util/comparator.cc + ${ROCKSDB_SOURCE_DIR}/util/compression.cc ${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc ${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc ${ROCKSDB_SOURCE_DIR}/util/crc32c.cc @@ -312,11 +318,13 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/slice.cc ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc ${ROCKSDB_SOURCE_DIR}/util/status.cc + ${ROCKSDB_SOURCE_DIR}/util/stderr_logger.cc ${ROCKSDB_SOURCE_DIR}/util/string_util.cc ${ROCKSDB_SOURCE_DIR}/util/thread_local.cc ${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc ${ROCKSDB_SOURCE_DIR}/util/xxhash.cc - ${ROCKSDB_SOURCE_DIR}/utilities/backupable/backupable_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/agg_merge/agg_merge.cc + ${ROCKSDB_SOURCE_DIR}/utilities/backup/backup_engine.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_compaction_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl.cc @@ -419,15 +427,6 @@ list(APPEND SOURCES "${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc" "${ROCKSDB_SOURCE_DIR}/env/io_posix.cc") -if(WITH_FOLLY_DISTRIBUTED_MUTEX) - list(APPEND SOURCES - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/detail/Futex.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/AtomicNotification.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/DistributedMutex.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/ParkingLot.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/WaitOptions.cpp") -endif() - add_library(_rocksdb ${SOURCES}) add_library(ch_contrib::rocksdb ALIAS _rocksdb) target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) diff --git a/contrib/rocksdb-cmake/build_version.cc b/contrib/rocksdb-cmake/build_version.cc index f9639da516f..d5ea56673e0 100644 --- a/contrib/rocksdb-cmake/build_version.cc +++ b/contrib/rocksdb-cmake/build_version.cc @@ -1,16 +1,33 @@ // Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved. -/// This file was edited for ClickHouse. #include #include "rocksdb/version.h" +#include "rocksdb/utilities/object_registry.h" #include "util/string_util.h" // The build script may replace these values with real values based // on whether or not GIT is available and the platform settings -static const std::string rocksdb_build_git_sha = "rocksdb_build_git_sha:0"; -static const std::string rocksdb_build_git_tag = "rocksdb_build_git_tag:master"; -static const std::string rocksdb_build_date = "rocksdb_build_date:2000-01-01"; +static const std::string rocksdb_build_git_sha = "rocksdb_build_git_sha:72438a678872544809393b831c7273794c074215"; +static const std::string rocksdb_build_git_tag = "rocksdb_build_git_tag:main"; +#define HAS_GIT_CHANGES 0 +#if HAS_GIT_CHANGES == 0 +// If HAS_GIT_CHANGES is 0, the GIT date is used. +// Use the time the branch/tag was last modified +static const std::string rocksdb_build_date = "rocksdb_build_date:2024-07-12 16:01:57"; +#else +// If HAS_GIT_CHANGES is > 0, the branch/tag has modifications. +// Use the time the build was created. +static const std::string rocksdb_build_date = "rocksdb_build_date:2024-07-13 17:15:50"; +#endif + +extern "C" { + +} // extern "C" + +std::unordered_map ROCKSDB_NAMESPACE::ObjectRegistry::builtins_ = { + +}; namespace ROCKSDB_NAMESPACE { static void AddProperty(std::unordered_map *props, const std::string& name) { @@ -39,12 +56,12 @@ const std::unordered_map& GetRocksBuildProperties() { } std::string GetRocksVersionAsString(bool with_patch) { - std::string version = ToString(ROCKSDB_MAJOR) + "." + ToString(ROCKSDB_MINOR); + std::string version = std::to_string(ROCKSDB_MAJOR) + "." + std::to_string(ROCKSDB_MINOR); if (with_patch) { - return version + "." + ToString(ROCKSDB_PATCH); + return version + "." + std::to_string(ROCKSDB_PATCH); } else { return version; - } + } } std::string GetRocksBuildInfoAsString(const std::string& program, bool verbose) { From 68aebce89f30eb7766c420d25e852e21e27dfe7d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 15 Jul 2024 14:41:37 +0800 Subject: [PATCH 027/496] fix failed uts --- src/Functions/printf.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index a890b886338..3efe854a53b 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -93,6 +93,8 @@ private: { T a = data[i]; s = fmt::sprintf(format, static_cast>(a)); + + res_chars.resize(curr_offset + s.size() + 1); memcpy(&res_chars[curr_offset], s.data(), s.size()); res_chars[curr_offset + s.size()] = 0; @@ -115,6 +117,8 @@ private: { auto a = concrete_column->getDataAt(i).toView(); s = fmt::sprintf(format, a); + + res_chars.resize(curr_offset + s.size() + 1); memcpy(&res_chars[curr_offset], s.data(), s.size()); res_chars[curr_offset + s.size()] = 0; From 77272c925dc15acc5fdd0260a1c0aab35b1df3c3 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 15:10:39 +0800 Subject: [PATCH 028/496] disable insertion and mutation --- docs/en/operations/settings/settings.md | 6 ++ src/Core/ServerSettings.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 7 ++ src/Interpreters/InterpreterDeleteQuery.cpp | 4 + src/Interpreters/InterpreterInsertQuery.cpp | 5 ++ .../__init__.py | 0 .../config/cluster.xml | 16 ++++ .../config/reading_node.xml | 3 + .../config/storage_policy.xml | 21 +++++ .../config/writing_node.xml | 3 + .../test.py | 84 +++++++++++++++++++ 11 files changed, 150 insertions(+) create mode 100644 tests/integration/test_disable_insertion_and_mutation/__init__.py create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/cluster.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..143ce836beb 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,9 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## disable_insertion_and_mutation + +Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries. + +Default value: `false`. diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..cf09874125d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,6 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ + M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c70a3397f4e..b9dd59909e6 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -46,6 +46,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNKNOWN_TABLE; extern const int UNKNOWN_DATABASE; + extern const int QUERY_IS_PROHIBITED; } @@ -191,6 +192,12 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) "to execute ALTERs of different types (replicated and non replicated) in single query"); } + if (mutation_commands.hasNonEmptyMutationCommands() || !partition_commands.empty()) + { + if (getContext()->getServerSettings().disable_insertion_and_mutation) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Mutations are prohibited"); + } + if (!alter_commands.empty()) { auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 39d5d9e9cef..5f3e3385148 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; + extern const int QUERY_IS_PROHIBITED; } @@ -50,6 +51,9 @@ BlockIO InterpreterDeleteQuery::execute() if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); + if (getContext()->getGlobalContext()->getServerSettings().disable_insertion_and_mutation) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Delete queries are prohibited"); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (database->shouldReplicateQuery(getContext(), query_ptr)) { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f396db70d21..b62a71de884 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,6 +44,7 @@ namespace ProfileEvents { extern const Event InsertQueriesWithSubqueries; extern const Event QueriesWithSubqueries; + extern const int QUERY_IS_PROHIBITED; } namespace DB @@ -406,6 +407,10 @@ BlockIO InterpreterInsertQuery::execute() StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); + if (getContext()->getServerSettings().disable_insertion_and_mutation + && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); + StoragePtr inner_table; if (const auto * mv = dynamic_cast(table.get())) inner_table = mv->getTargetTable(); diff --git a/tests/integration/test_disable_insertion_and_mutation/__init__.py b/tests/integration/test_disable_insertion_and_mutation/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml b/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml new file mode 100644 index 00000000000..17782a77679 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml @@ -0,0 +1,16 @@ + + + + + + writing_node + 9000 + + + reading_node + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml b/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml new file mode 100644 index 00000000000..becabce8a44 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml @@ -0,0 +1,3 @@ + + true + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml new file mode 100644 index 00000000000..cec96cfcc1a --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml @@ -0,0 +1,21 @@ + + + + + s3_with_keeper + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ s3_with_keeper +
+
+
+
+
+
\ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml b/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml new file mode 100644 index 00000000000..0737af7afc7 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml @@ -0,0 +1,3 @@ + + false + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py new file mode 100644 index 00000000000..5234ae9c57c --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -0,0 +1,84 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +import time + +cluster = ClickHouseCluster(__file__) + +writing_node = cluster.add_instance( + "writing_node", + main_configs=["config/writing_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) +reading_node = cluster.add_instance( + "reading_node", + main_configs=["config/reading_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 2}, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_disable_insertion_and_mutation(started_cluster): + writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) SETTINGS storage_policy='s3_with_keeper' """) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("INSERT INTO my_table VALUES (1, 'hello')") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER TABLE my_table delete where 1") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER table my_table update key = 1 where 1") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER TABLE my_table drop partition 0") + ) + + reading_node.query("SELECT * from my_table"); + writing_node.query("INSERT INTO my_table VALUES (1, 'hello')") + writing_node.query("ALTER TABLE my_table delete where 1") + writing_node.query("ALTER table my_table update value = 'no hello' where 1") + + reading_node.query("ALTER TABLE my_table ADD COLUMN new_column UInt64") + writing_node.query("SELECT new_column from my_table") + reading_node.query("SELECT new_column from my_table") + + reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") + + assert( + "new_column\tString" + in reading_node.query("DESC my_table") + ) + + assert( + "new_column\tString" + in writing_node.query("DESC my_table") + ) From fc29ac7891eddd3a714f5af574c71040f91f451d Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 17:06:37 +0800 Subject: [PATCH 029/496] add error extern to fix compile error --- src/Interpreters/InterpreterInsertQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b62a71de884..c01b2196ac9 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -56,6 +56,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; + extern const int QUERY_IS_PROHIBITED; } InterpreterInsertQuery::InterpreterInsertQuery( From ccba078da10bed8d42e821f8bcdd47f448d198a0 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 09:46:31 +0000 Subject: [PATCH 030/496] change storage policy to default --- .../config/storage_policy.xml | 21 ------------------- .../test.py | 6 +++--- 2 files changed, 3 insertions(+), 24 deletions(-) delete mode 100644 tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml diff --git a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml deleted file mode 100644 index cec96cfcc1a..00000000000 --- a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - s3_with_keeper - http://minio1:9001/root/data/ - minio - minio123 - - - - - -
- s3_with_keeper -
-
-
-
-
-
\ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index 5234ae9c57c..f098f130d2b 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) writing_node = cluster.add_instance( "writing_node", - main_configs=["config/writing_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + main_configs=["config/writing_node.xml", "config/cluster.xml"], with_zookeeper=True, with_minio=True, stay_alive=True, @@ -15,7 +15,7 @@ writing_node = cluster.add_instance( ) reading_node = cluster.add_instance( "reading_node", - main_configs=["config/reading_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + main_configs=["config/reading_node.xml", "config/cluster.xml"], with_zookeeper=True, with_minio=True, stay_alive=True, @@ -35,7 +35,7 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): - writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) SETTINGS storage_policy='s3_with_keeper' """) + writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) assert ( "QUERY_IS_PROHIBITED" From b2466466d46ab1828b67693edf520d52ff45bc65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 15 Jul 2024 14:28:12 +0200 Subject: [PATCH 031/496] Rename Context::getSettings() --- programs/local/LocalServer.cpp | 2 +- src/Analyzer/QueryTreeBuilder.cpp | 2 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- src/Analyzer/Utils.cpp | 2 +- src/Bridge/IBridge.cpp | 2 +- src/Client/ClientBase.cpp | 6 +++--- src/Databases/DatabaseDictionary.cpp | 4 ++-- src/Databases/DatabaseOnDisk.cpp | 4 ++-- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Functions/formatQuery.cpp | 2 +- src/Functions/hasColumnInTable.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 5 ++--- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 2 +- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/interpretSubquery.cpp | 2 +- src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- .../QueryPlan/Optimizations/optimizeReadInOrder.cpp | 2 +- src/Server/MySQLHandler.cpp | 2 +- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 2 +- src/Storages/StorageURL.cpp | 4 ++-- src/Storages/StorageView.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 2 +- src/TableFunctions/Hive/TableFunctionHive.cpp | 2 +- 30 files changed, 37 insertions(+), 38 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 46b543e49e9..a7265ef0de4 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -80,7 +80,7 @@ namespace ErrorCodes void applySettingsOverridesForLocal(ContextMutablePtr context) { - Settings settings = context->getSettings(); + Settings settings = context->getSettingsCopy(); settings.allow_introspection_functions = true; settings.storage_file_read_method = LocalFSReadMethod::mmap; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index a62b6e56ac5..0a732a3b3b3 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -237,7 +237,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q /// Remove global settings limit and offset if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset) { - Settings settings = updated_context->getSettings(); + Settings settings = updated_context->getSettingsCopy(); limit = settings.limit; offset = settings.offset; settings.limit = 0; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 5f7b06231d9..92618dfe346 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -503,7 +503,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden ProfileEvents::increment(ProfileEvents::ScalarSubqueriesCacheMiss); auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 1; subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index d10bbd9bd23..e5f372b7368 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -867,7 +867,7 @@ void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context) * max_rows_in_join, max_bytes_in_join, join_overflow_mode, * which are checked separately (in the Set, Join objects). */ - Settings subquery_settings = mutable_context->getSettings(); + Settings subquery_settings = mutable_context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index de48a4f2b84..5682a28f899 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -232,7 +232,7 @@ int IBridge::main(const std::vector & /*args*/) auto context = Context::createGlobal(shared_context.get()); context->makeGlobalContext(); - auto settings = context->getSettings(); + auto settings = context->getSettingsCopy(); settings.set("http_max_field_value_size", http_max_field_value_size); context->setSettings(settings); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d472ba99b9..2dc603a307f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -723,7 +723,7 @@ void ClientBase::initLogsOutputStream() void ClientBase::adjustSettings() { - Settings settings = global_context->getSettings(); + Settings settings = global_context->getSettingsCopy(); /// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles) @@ -931,7 +931,7 @@ bool ClientBase::isSyncInsertWithData(const ASTInsertQuery & insert_query, const if (!insert_query.data) return false; - auto settings = context->getSettings(); + auto settings = context->getSettingsCopy(); if (insert_query.settings_ast) settings.applyChanges(insert_query.settings_ast->as()->changes); @@ -2696,7 +2696,7 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name) if (!getClientConfiguration().has("log_comment")) { - Settings settings = global_context->getSettings(); + Settings settings = global_context->getSettingsCopy(); /// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]" settings.log_comment = fs::absolute(fs::path(file_name)); global_context->setSettings(settings); diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 52196e75c4a..a9569408814 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -111,7 +111,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; @@ -133,7 +133,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) buffer << " COMMENT " << backQuote(comment_value); } - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ParserCreateQuery parser; return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 07a250e72c7..261a917c595 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -523,7 +523,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const { ASTPtr ast; - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); { std::lock_guard lock(mutex); auto database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"; @@ -722,7 +722,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( return nullptr; } - auto settings = local_context->getSettingsRef(); + const auto & settings = local_context->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 2c342755337..04b4070d5af 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -89,7 +89,7 @@ static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; static ContextMutablePtr createQueryContext(ContextPtr context) { - Settings new_query_settings = context->getSettings(); + Settings new_query_settings = context->getSettingsCopy(); new_query_settings.insert_allow_materialized_columns = true; /// To avoid call AST::format diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index d10b3f9a5b7..f7f7e4b5bcb 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -39,7 +39,7 @@ public: FunctionFormatQuery(ContextPtr context, String name_, OutputFormatting output_formatting_, ErrorHandling error_handling_) : name(name_), output_formatting(output_formatting_), error_handling(error_handling_) { - const Settings & settings = context->getSettings(); + const Settings & settings = context->getSettingsRef(); max_query_size = settings.max_query_size; max_parser_depth = settings.max_parser_depth; max_parser_backtracks = settings.max_parser_backtracks; diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 00714997b4a..cc496270b01 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -143,7 +143,7 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a /* cluster_name= */ "", /* password= */ "" }; - auto cluster = std::make_shared(getContext()->getSettings(), host_names, params); + auto cluster = std::make_shared(getContext()->getSettingsRef(), host_names, params); // FIXME this (probably) needs a non-constant access to query context, // because it might initialized a storage. Ideally, the tables required diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..3a88e0ccfe1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2267,7 +2267,7 @@ bool Context::displaySecretsInShowAndSelect() const return shared->server_settings.display_secrets_in_show_and_select; } -Settings Context::getSettings() const +Settings Context::getSettingsCopy() const { SharedLockGuard lock(mutex); return *settings; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 284cac50769..61095e53a17 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -830,7 +830,8 @@ public: void setMacros(std::unique_ptr && macros); bool displaySecretsInShowAndSelect() const; - Settings getSettings() const; + Settings getSettingsCopy() const; + const Settings & getSettingsRef() const { return *settings; } void setSettings(const Settings & settings_); /// Set settings by name. @@ -955,8 +956,6 @@ public: void makeSessionContext(); void makeGlobalContext(); - const Settings & getSettingsRef() const { return *settings; } - void setProgressCallback(ProgressCallback callback); /// Used in executeQuery() to pass it to the QueryPipeline. ProgressCallback getProgressCallback() const; diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 4bd1c47d5a0..1ca8c40460c 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -74,7 +74,7 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data) static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqueriesMatcher::Data & data) { auto subquery_context = Context::createCopy(data.getContext()); - Settings subquery_settings = data.getContext()->getSettings(); + Settings subquery_settings = data.getContext()->getSettingsCopy(); subquery_settings.max_result_rows = 1; subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 16d0eb71278..1a4c02bdebb 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -171,7 +171,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( PreparedSetsPtr prepared_sets_, bool is_create_parameterized_view_) : WithContext(context_) - , query(query_), settings(getContext()->getSettings()) + , query(query_), settings(getContext()->getSettingsRef()) , subquery_depth(subquery_depth_) , syntax(syntax_analyzer_result_) , is_create_parameterized_view(is_create_parameterized_view_) @@ -983,7 +983,7 @@ static std::shared_ptr tryCreateJoin( algorithm == JoinAlgorithm::PARALLEL_HASH || algorithm == JoinAlgorithm::DEFAULT) { - const auto & settings = context->getSettings(); + const auto & settings = context->getSettingsRef(); if (analyzed_join->allowParallelHashJoin()) return std::make_shared( diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index ef298d4d45a..dffa0cbaa5b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -469,7 +469,7 @@ BlockIO InterpreterInsertQuery::execute() * to avoid unnecessary squashing. */ - Settings new_settings = getContext()->getSettings(); + Settings new_settings = getContext()->getSettingsCopy(); new_settings.max_threads = std::max(1, settings.max_insert_threads); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cd91f9532b9..cb3c478dbb1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -253,7 +253,7 @@ namespace ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 0de2bf9cb1f..c5226107f8d 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -308,7 +308,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se if (tables_with_columns.size() < 2) return {}; - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); MultiEnum join_algorithm = settings.join_algorithm; bool try_use_direct_join = join_algorithm.isSet(JoinAlgorithm::DIRECT) || join_algorithm.isSet(JoinAlgorithm::DEFAULT); auto table_join = std::make_shared(settings, context->getGlobalTemporaryVolume(), context->getTempDataOnDisk()); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 5b07852d9e3..271e23a7288 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -657,7 +657,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even { if (auto ctx = context.lock()) { - res.query_settings = std::make_shared(ctx->getSettings()); + res.query_settings = std::make_shared(ctx->getSettingsRef()); res.current_database = ctx->getCurrentDatabase(); } } diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 340f6d1d805..909875b99a0 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -62,7 +62,7 @@ std::shared_ptr interpretSubquery( * which are checked separately (in the Set, Join objects). */ auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c23d717d52f..e467c358d1d 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -406,7 +406,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx { const Block & header = getPort().getHeader(); const IDataType & type = *header.getByPosition(column_idx).type; - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); /// Advance the token iterator until the start of the column expression readUntilTheEndOfRowAndReTokenize(column_idx); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 28eb4da2e17..415a6a11999 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1057,7 +1057,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, } auto context = read_from_merge_tree->getContext(); - const auto & settings = context->getSettings(); + const auto & settings = context->getSettingsRef(); if (!settings.optimize_read_in_window_order || (settings.optimize_read_in_order && settings.query_plan_read_in_order) || context->getSettingsRef().allow_experimental_analyzer) { return 0; diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index b6d795b1e69..3deb09bae88 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -473,7 +473,7 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool binary_protocol) query_context->setCurrentQueryId(fmt::format("mysql:{}:{}", connection_id, toString(UUIDHelpers::generateV4()))); /// --- Workaround for Bug 56173. Can be removed when the analyzer is on by default. - auto settings = query_context->getSettings(); + auto settings = query_context->getSettingsCopy(); settings.prefer_column_name_to_alias = true; query_context->setSettings(settings); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..c287fc817eb 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -283,7 +283,7 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c auto pools = createPoolsForAddresses(addresses, pool_factory, storage.log); - const auto settings = storage.getContext()->getSettings(); + const auto & settings = storage.getContext()->getSettingsRef(); return std::make_shared(std::move(pools), settings.load_balancing, settings.distributed_replica_error_half_life.totalSeconds(), diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 38869aebaa5..4ee68580d3f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -34,7 +34,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( , write_settings(write_settings_) { MergeTreeWriterSettings writer_settings( - data_part->storage.getContext()->getSettings(), + data_part->storage.getContext()->getSettingsRef(), write_settings, storage_settings, data_part->index_granularity_info.mark_type.adaptive, diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index c167ac87317..05cd77dcd40 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -23,7 +23,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeIndexGranularityInfo * index_granularity_info) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { - const auto & global_settings = data_part->storage.getContext()->getSettings(); + const auto & global_settings = data_part->storage.getContext()->getSettingsRef(); MergeTreeWriterSettings writer_settings( global_settings, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9cec8c75ebe..731bd7ec3d3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -462,7 +462,7 @@ std::pair> StorageURLSource: setCredentials(credentials, request_uri); - const auto settings = context_->getSettings(); + const auto & settings = context_->getSettingsRef(); auto proxy_config = getProxyConfiguration(request_uri.getScheme()); @@ -1324,7 +1324,7 @@ std::optional IStorageURLBase::tryGetLastModificationTime( const Poco::Net::HTTPBasicCredentials & credentials, const ContextPtr & context) { - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); auto uri = Poco::URI(url); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 5f768bce978..929896e3246 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -97,7 +97,7 @@ bool hasJoin(const ASTSelectWithUnionQuery & ast) ContextPtr getViewContext(ContextPtr context, const StorageSnapshotPtr & storage_snapshot) { auto view_context = storage_snapshot->metadata->getSQLSecurityOverriddenContext(context); - Settings view_settings = view_context->getSettings(); + Settings view_settings = view_context->getSettingsCopy(); view_settings.max_result_rows = 0; view_settings.max_result_bytes = 0; view_settings.extremes = false; diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 56071abaa95..9d23f132759 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -65,7 +65,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), /// since this is a service query and should not lead to query failure. { - Settings new_settings = new_context->getSettings(); + Settings new_settings = new_context->getSettingsCopy(); new_settings.max_result_rows = 0; new_settings.max_result_bytes = 0; new_context->setSettings(new_settings); diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index 80494dbe5a8..759807d7a4f 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -93,7 +93,7 @@ StoragePtr TableFunctionHive::executeImpl( ColumnsDescription /*cached_columns_*/, bool /*is_insert_query*/) const { - const Settings & settings = context_->getSettings(); + const Settings & settings = context_->getSettingsRef(); ParserExpression partition_by_parser; ASTPtr partition_by_ast = parseQuery( partition_by_parser, From 908f5899ddfdc701df5e9e6189760431e88b6695 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 17:28:33 +0200 Subject: [PATCH 032/496] Add settings to replace external engines to Null --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 3 +- src/Interpreters/InterpreterCreateQuery.cpp | 40 +++++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 52fa28a4481..7bf97896357 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,6 +891,7 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ + M(Bool, restore_replace_external_engine_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9b72209103..a23d9d17da2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,8 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } + + void setNullTableEngine(ASTStorage &storage) + { + auto engine_ast = std::make_shared(); + engine_ast->name = "Null"; + engine_ast->no_empty_args = true; + storage.set(storage.engine, engine_ast); + } } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const @@ -1000,6 +1008,38 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. if (!create.storage->engine) setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); + /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to + /// Null table engine. + else (create.storage->engine == "AzureBlobStorage" || + create.storage->engine == "AzureQueue" || + create.storage->engine == "COSN" || + create.storage->engine == "DeltaLake" || + create.storage->engine == "Dictionary" || + create.storage->engine == "Executable" || + create.storage->engine == "ExecutablePool" || + create.storage->engine == "ExternalDistributed" || + create.storage->engine == "File" || + create.storage->engine == "Hudi" || + create.storage->engine == "Iceberg" || + create.storage->engine == "JDBC" || + create.storage->engine == "Kafka" || + create.storage->engine == "MaterializedPostgreSQL" || + create.storage->engine == "MongoDB" || + create.storage->engine == "MySQL" || + create.storage->engine == "NATS" || + create.storage->engine == "ODBC" || + create.storage->engine == "OSS" || + create.storage->engine == "PostgreSQL" || + create.storage->engine == "RabbitMQ" || + create.storage->engine == "Redis" || + create.storage->engine == "S3" || + create.storage->engine == "S3Queue" || + create.storage->engine == "TinyLog" || + create.storage->engine == "URL") + { + if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) + setNullTableEngine(*create.storage) + } return; } From 7d70968db3527d894bc6c02d51dc70f932f7eacd Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 18:47:04 +0200 Subject: [PATCH 033/496] try fix --- src/Interpreters/InterpreterCreateQuery.cpp | 52 ++++++++++----------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f8696caebe7..9eb13a29af7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1010,32 +1010,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else (create.storage->engine == "AzureBlobStorage" || - create.storage->engine == "AzureQueue" || - create.storage->engine == "COSN" || - create.storage->engine == "DeltaLake" || - create.storage->engine == "Dictionary" || - create.storage->engine == "Executable" || - create.storage->engine == "ExecutablePool" || - create.storage->engine == "ExternalDistributed" || - create.storage->engine == "File" || - create.storage->engine == "Hudi" || - create.storage->engine == "Iceberg" || - create.storage->engine == "JDBC" || - create.storage->engine == "Kafka" || - create.storage->engine == "MaterializedPostgreSQL" || - create.storage->engine == "MongoDB" || - create.storage->engine == "MySQL" || - create.storage->engine == "NATS" || - create.storage->engine == "ODBC" || - create.storage->engine == "OSS" || - create.storage->engine == "PostgreSQL" || - create.storage->engine == "RabbitMQ" || - create.storage->engine == "Redis" || - create.storage->engine == "S3" || - create.storage->engine == "S3Queue" || - create.storage->engine == "TinyLog" || - create.storage->engine == "URL") + else (create.storage->engine->name == "AzureBlobStorage" || + create.storage->engine->name == "AzureQueue" || + create.storage->engine->name == "COSN" || + create.storage->engine->name == "DeltaLake" || + create.storage->engine->name == "Dictionary" || + create.storage->engine->name == "Executable" || + create.storage->engine->name == "ExecutablePool" || + create.storage->engine->name == "ExternalDistributed" || + create.storage->engine->name == "File" || + create.storage->engine->name == "Hudi" || + create.storage->engine->name == "Iceberg" || + create.storage->engine->name == "JDBC" || + create.storage->engine->name == "Kafka" || + create.storage->engine->name == "MaterializedPostgreSQL" || + create.storage->engine->name == "MongoDB" || + create.storage->engine->name == "MySQL" || + create.storage->engine->name == "NATS" || + create.storage->engine->name == "ODBC" || + create.storage->engine->name == "OSS" || + create.storage->engine->name == "PostgreSQL" || + create.storage->engine->name == "RabbitMQ" || + create.storage->engine->name == "Redis" || + create.storage->engine->name == "S3" || + create.storage->engine->name == "S3Queue" || + create.storage->engine->name == "TinyLog" || + create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) setNullTableEngine(*create.storage) From fc49b1b75f9b075f28cdc4b7eeb768339bb1ebd5 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 19:02:21 +0200 Subject: [PATCH 034/496] semicolon --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9eb13a29af7..94230f0e7d1 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1038,7 +1038,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) - setNullTableEngine(*create.storage) + setNullTableEngine(*create.storage); } return; } From 083e4b17db62121d6905c35480c3a462dc26477b Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 09:34:52 +0800 Subject: [PATCH 035/496] trigger CI From 478616de3d03495cf8c324da9464a9807b51ba41 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Jul 2024 10:54:39 +0000 Subject: [PATCH 036/496] forgot --- src/Interpreters/InterpreterCreateQuery.cpp | 52 ++++++++++----------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 94230f0e7d1..3b23c6899e9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1010,32 +1010,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else (create.storage->engine->name == "AzureBlobStorage" || - create.storage->engine->name == "AzureQueue" || - create.storage->engine->name == "COSN" || - create.storage->engine->name == "DeltaLake" || - create.storage->engine->name == "Dictionary" || - create.storage->engine->name == "Executable" || - create.storage->engine->name == "ExecutablePool" || - create.storage->engine->name == "ExternalDistributed" || - create.storage->engine->name == "File" || - create.storage->engine->name == "Hudi" || - create.storage->engine->name == "Iceberg" || - create.storage->engine->name == "JDBC" || - create.storage->engine->name == "Kafka" || - create.storage->engine->name == "MaterializedPostgreSQL" || - create.storage->engine->name == "MongoDB" || - create.storage->engine->name == "MySQL" || - create.storage->engine->name == "NATS" || - create.storage->engine->name == "ODBC" || - create.storage->engine->name == "OSS" || - create.storage->engine->name == "PostgreSQL" || - create.storage->engine->name == "RabbitMQ" || - create.storage->engine->name == "Redis" || - create.storage->engine->name == "S3" || - create.storage->engine->name == "S3Queue" || - create.storage->engine->name == "TinyLog" || - create.storage->engine->name == "URL") + else if (create.storage->engine->name == "AzureBlobStorage" || + create.storage->engine->name == "AzureQueue" || + create.storage->engine->name == "COSN" || + create.storage->engine->name == "DeltaLake" || + create.storage->engine->name == "Dictionary" || + create.storage->engine->name == "Executable" || + create.storage->engine->name == "ExecutablePool" || + create.storage->engine->name == "ExternalDistributed" || + create.storage->engine->name == "File" || + create.storage->engine->name == "Hudi" || + create.storage->engine->name == "Iceberg" || + create.storage->engine->name == "JDBC" || + create.storage->engine->name == "Kafka" || + create.storage->engine->name == "MaterializedPostgreSQL" || + create.storage->engine->name == "MongoDB" || + create.storage->engine->name == "MySQL" || + create.storage->engine->name == "NATS" || + create.storage->engine->name == "ODBC" || + create.storage->engine->name == "OSS" || + create.storage->engine->name == "PostgreSQL" || + create.storage->engine->name == "RabbitMQ" || + create.storage->engine->name == "Redis" || + create.storage->engine->name == "S3" || + create.storage->engine->name == "S3Queue" || + create.storage->engine->name == "TinyLog" || + create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) setNullTableEngine(*create.storage); From b6a790124cd670749b4c504f58a4854307bf7d83 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 20:16:47 +0000 Subject: [PATCH 037/496] Handling parallel replicas protocol with priority for async communication --- src/Processors/IProcessor.h | 2 + src/Processors/Sources/RemoteSource.cpp | 23 +++++++++++ src/Processors/Sources/RemoteSource.h | 3 ++ src/QueryPipeline/RemoteQueryExecutor.cpp | 38 +++++++++++++++++-- src/QueryPipeline/RemoteQueryExecutor.h | 4 +- .../RemoteQueryExecutorReadContext.h | 2 + 6 files changed, 67 insertions(+), 5 deletions(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 02f7b6b3d12..358983a2179 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,6 +221,8 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } + virtual void asyncJobReady() {} + /** You must call this method if 'prepare' returned ExpandPipeline. * This method cannot access any port, but it can create new ports for current processor. * diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..f1d47f69782 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -89,6 +89,12 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { + if (async_immediate_work.exchange(false)) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work was true"); + return; + } + /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. /// See issue: https://github.com/ClickHouse/ClickHouse/issues/60844 @@ -101,6 +107,23 @@ void RemoteSource::work() ISource::work(); } +void RemoteSource::asyncJobReady() +{ + chassert(async_read); + + if (!was_query_sent) + return; + + auto res = query_executor->readAsync(/*probe=*/true); + if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is {}", async_immediate_work); + work(); + async_immediate_work = true; + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is true"); + } +} + std::optional RemoteSource::tryGenerate() { /// onCancel() will do the cancel if the query was sent. diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 052567bc261..fa04985f101 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -32,6 +32,8 @@ public: int schedule() override { return fd; } + void asyncJobReady() override; + void setStorageLimits(const std::shared_ptr & storage_limits_) override; protected: @@ -52,6 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; + std::atomic_bool async_immediate_work{false}; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b08f2002f64..3ca05b53417 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet_type_only) { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) @@ -486,7 +486,21 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "was_cancelled"); return ReadResult(Block()); + } + + if (has_postponed_packet) + { + has_postponed_packet = false; + auto read_result = processPacket(read_context->getPacket()); + if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) + return read_result; + + if (got_duplicated_part_uuids) + break; + } read_context->resume(); @@ -506,12 +520,28 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() /// Check if packet is not ready yet. if (read_context->isInProgress()) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "read_context still in progress"); return ReadResult(read_context->getFileDescriptor()); + } - auto anything = processPacket(read_context->getPacket()); + const auto packet_type = read_context->getPacketType(); + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Packet type: {}", packet_type); - if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken) - return anything; + if (check_packet_type_only) + { + has_postponed_packet = true; + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest + || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + return ReadResult(ReadResult::Type::ParallelReplicasToken); + } + return ReadResult(ReadResult::Type::Nothing); + } + + auto read_result = processPacket(read_context->getPacket()); + if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) + return read_result; if (got_duplicated_part_uuids) break; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 04a59cc3b7e..6849c3e0a07 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -183,7 +183,7 @@ public: ReadResult read(); /// Async variant of read. Returns ready block or file descriptor which may be used for polling. - ReadResult readAsync(); + ReadResult readAsync(bool check_packet_type_only = false); /// Receive all remain packets and finish query. /// It should be cancelled after read returned empty block. @@ -303,6 +303,8 @@ private: */ bool got_duplicated_part_uuids = false; + bool has_postponed_packet = false; + /// Parts uuids, collected from remote replicas std::vector duplicated_part_uuids; diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index b8aa8bb9111..c054e75f6f1 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -39,6 +39,8 @@ public: Packet getPacket() { return std::move(packet); } + UInt64 getPacketType() const { return packet.type; } + private: bool checkTimeout(bool blocking = false); From a7310e51939ad6053d6ab94b07f0171457e5d779 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 19:32:27 +0200 Subject: [PATCH 038/496] Ignore async_load_databases for ATTACH query It is quite odd that when ATTACH finishes the tables may not be exists, due to async_load_databases. For server startup it makes total sense, but not for queries. Plus, you can execute queries in parallel if you want to make it faster. Note, that server startup does not uses this code, see loadMetadata.cpp. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterCreateQuery.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 84d7f0a587c..1d8d885b216 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -362,18 +362,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) TablesLoader loader{getContext()->getGlobalContext(), {{database_name, database}}, mode}; auto load_tasks = loader.loadTablesAsync(); auto startup_tasks = loader.startupTablesAsync(); - if (getContext()->getGlobalContext()->getServerSettings().async_load_databases) - { - scheduleLoad(load_tasks); - scheduleLoad(startup_tasks); - } - else - { - /// First prioritize, schedule and wait all the load table tasks - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), load_tasks); - /// Only then prioritize, schedule and wait all the startup tasks - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_tasks); - } + /// First prioritize, schedule and wait all the load table tasks + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), load_tasks); + /// Only then prioritize, schedule and wait all the startup tasks + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_tasks); } } catch (...) From f30d35ae2926948f1e6a268917113e757df4e2df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 19:34:18 +0200 Subject: [PATCH 039/496] Revert "Merge pull request #65571 from ClickHouse/fix-flaky-test-4" Reverts: https://github.com/ClickHouse/ClickHouse/pull/65571 This reverts commit da9a34ea46b504881ffe5aa605c933106862ba25, reversing changes made to cbdb9833f207d4b0e35ad09cf4757f5d5b506b77. Signed-off-by: Azat Khuzhin --- .../0_stateless/01254_dict_load_after_detach_attach.reference | 2 +- .../queries/0_stateless/01254_dict_load_after_detach_attach.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference index 9c2c59f6379..2f2d638a294 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference @@ -1,4 +1,4 @@ -NOT_LOADED +0 NOT_LOADED 0 LOADED 10 1 LOADED diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql index 11473c6ce32..ef9e940df8b 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql @@ -12,7 +12,7 @@ LAYOUT(FLAT()); DETACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; ATTACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; -SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict')::Nullable(String), 'NOT_LOADED'); +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SYSTEM RELOAD DICTIONARY dict; SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT dictGetUInt64('dict', 'val', toUInt64(0)); From df636100d3643031ee8d2102b737e992a6e44a40 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:00:53 +0200 Subject: [PATCH 040/496] add entry in documentation for use_same_password_for_base_backup --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index fc861e25e9f..248fdbc156f 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -85,6 +85,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `use_same_s3_credentials_for_base_backup`: whether base backup to S3 should inherit credentials from the query. Only works with `S3`. + - `use_same_password_for_base_backup`: whether base backup archive should inherit the password from the query. - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` From d8c68a27744dca3f68e4de08036148c2ecfd2ed6 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:00:53 +0200 Subject: [PATCH 041/496] working incremental password protected backups --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupImpl.cpp | 13 +++++++++++-- src/Backups/BackupImpl.h | 7 +++++-- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 +++ src/Backups/BackupsWorker.cpp | 2 ++ src/Backups/RestoreSettings.cpp | 1 + src/Backups/RestoreSettings.h | 3 +++ .../registerBackupEngineAzureBlobStorage.cpp | 6 ++++-- src/Backups/registerBackupEngineS3.cpp | 6 ++++-- src/Backups/registerBackupEnginesFileAndDisk.cpp | 6 ++++-- 11 files changed, 39 insertions(+), 10 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index e13a9a12ca2..807b8516d49 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -41,6 +41,7 @@ public: bool allow_s3_native_copy = true; bool allow_azure_native_copy = true; bool use_same_s3_credentials_for_base_backup = false; + bool use_same_password_for_base_backup = false; bool azure_attempt_to_create_container = true; ReadSettings read_settings; WriteSettings write_settings; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 3f972c36e47..23f067a62f5 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -92,7 +92,8 @@ BackupImpl::BackupImpl( std::shared_ptr reader_, const ContextPtr & context_, bool is_internal_backup_, - bool use_same_s3_credentials_for_base_backup_) + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_) : backup_info(backup_info_) , backup_name_for_logging(backup_info.toStringForLogging()) , use_archive(!archive_params_.archive_name.empty()) @@ -104,6 +105,7 @@ BackupImpl::BackupImpl( , version(INITIAL_BACKUP_VERSION) , base_backup_info(base_backup_info_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) + , use_same_password_for_base_backup(use_same_password_for_base_backup_) , log(getLogger("BackupImpl")) { open(); @@ -120,7 +122,8 @@ BackupImpl::BackupImpl( const std::shared_ptr & coordination_, const std::optional & backup_uuid_, bool deduplicate_files_, - bool use_same_s3_credentials_for_base_backup_) + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_) : backup_info(backup_info_) , backup_name_for_logging(backup_info.toStringForLogging()) , use_archive(!archive_params_.archive_name.empty()) @@ -135,6 +138,7 @@ BackupImpl::BackupImpl( , base_backup_info(base_backup_info_) , deduplicate_files(deduplicate_files_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) + , use_same_password_for_base_backup(use_same_password_for_base_backup_) , log(getLogger("BackupImpl")) { open(); @@ -258,6 +262,11 @@ std::shared_ptr BackupImpl::getBaseBackupUnlocked() const params.is_internal_backup = is_internal_backup; /// use_same_s3_credentials_for_base_backup should be inherited for base backups params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup; + /// use_same_password_for_base_backup should be inherited for base backups + params.use_same_password_for_base_backup = use_same_password_for_base_backup; + + if (params.use_same_password_for_base_backup) + params.password = archive_params.password; base_backup = BackupFactory::instance().createBackup(params); diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 2b27e2ab090..d7846104c4c 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -41,7 +41,8 @@ public: std::shared_ptr reader_, const ContextPtr & context_, bool is_internal_backup_, - bool use_same_s3_credentials_for_base_backup_); + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_); BackupImpl( const BackupInfo & backup_info_, @@ -53,7 +54,8 @@ public: const std::shared_ptr & coordination_, const std::optional & backup_uuid_, bool deduplicate_files_, - bool use_same_s3_credentials_for_base_backup_); + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_); ~BackupImpl() override; @@ -153,6 +155,7 @@ private: bool writing_finalized = false; bool deduplicate_files = true; bool use_same_s3_credentials_for_base_backup = false; + bool use_same_password_for_base_backup = false; const LoggerPtr log; }; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index e33880f88e3..37ddd344001 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes M(Bool, allow_s3_native_copy) \ M(Bool, allow_azure_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ + M(Bool, use_same_password_for_base_backup) \ M(Bool, azure_attempt_to_create_container) \ M(Bool, read_from_filesystem_cache) \ M(UInt64, shard_num) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index a6c4d5d7181..d8f48f6e1ac 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -50,6 +50,9 @@ struct BackupSettings /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; + /// Wheter base backup archive should be unlocked using the same password as the incremental archive + bool use_same_password_for_base_backup = false; + /// Whether a new Azure container should be created if it does not exist (requires permissions at storage account level) bool azure_attempt_to_create_container = true; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 15a7d7c1eca..0614fb2da01 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -601,6 +601,7 @@ void BackupsWorker::doBackup( 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.use_same_password_for_base_backup = backup_settings.use_same_password_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); backup_create_params.write_settings = getWriteSettingsForBackup(context); @@ -912,6 +913,7 @@ void BackupsWorker::doRestore( backup_open_params.password = restore_settings.password; backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; backup_open_params.use_same_s3_credentials_for_base_backup = restore_settings.use_same_s3_credentials_for_base_backup; + backup_open_params.use_same_password_for_base_backup = restore_settings.use_same_password_for_base_backup; backup_open_params.read_settings = getReadSettingsForRestore(context); backup_open_params.write_settings = getWriteSettingsForRestore(context); backup_open_params.is_internal_backup = restore_settings.internal; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 7bbfd9ed751..a974fc11d00 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -164,6 +164,7 @@ namespace M(RestoreUDFCreationMode, create_function) \ M(Bool, allow_s3_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ + M(Bool, use_same_password_for_base_backup) \ M(Bool, restore_broken_parts_as_detached) \ M(Bool, internal) \ M(String, host_id) \ diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 06ecbc80aef..0fe5ee1a4bf 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -113,6 +113,9 @@ struct RestoreSettings /// Whether base backup from S3 should inherit credentials from the RESTORE query. bool use_same_s3_credentials_for_base_backup = false; + /// Wheter base backup archive should be unlocked using the same password as the incremental archive + bool use_same_password_for_base_backup = false; + /// If it's true RESTORE won't stop on broken parts while restoring, instead they will be restored as detached parts /// to the `detached` folder with names starting with `broken-from-backup'. bool restore_broken_parts_as_detached = false; diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 626df99b00c..45f0386375a 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -141,7 +141,8 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) reader, params.context, params.is_internal_backup, - /* use_same_s3_credentials_for_base_backup*/ false); + /* use_same_s3_credentials_for_base_backup*/ false, + params.use_same_password_for_base_backup); } else { @@ -164,7 +165,8 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - /* use_same_s3_credentials_for_base_backup */ false); + /* use_same_s3_credentials_for_base_backup */ false, + params.use_same_password_for_base_backup); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "AzureBlobStorage support is disabled"); diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 59ed9506af0..79e3e945557 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -120,7 +120,8 @@ void registerBackupEngineS3(BackupFactory & factory) reader, params.context, params.is_internal_backup, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } else { @@ -144,7 +145,8 @@ void registerBackupEngineS3(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "S3 support is disabled"); diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 35263d39cba..c486f79a77a 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -178,7 +178,8 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) reader, params.context, params.is_internal_backup, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } else { @@ -197,7 +198,8 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } }; From 74de7833b8d429edcac9c4735143fc92349abdaf Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:34:53 +0200 Subject: [PATCH 042/496] fix typo --- src/Backups/BackupSettings.h | 2 +- src/Backups/RestoreSettings.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index d8f48f6e1ac..0abeb897db4 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -50,7 +50,7 @@ struct BackupSettings /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; - /// Wheter base backup archive should be unlocked using the same password as the incremental archive + /// Whether base backup archive should be unlocked using the same password as the incremental archive bool use_same_password_for_base_backup = false; /// Whether a new Azure container should be created if it does not exist (requires permissions at storage account level) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 0fe5ee1a4bf..fe07a0a7208 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -113,7 +113,7 @@ struct RestoreSettings /// Whether base backup from S3 should inherit credentials from the RESTORE query. bool use_same_s3_credentials_for_base_backup = false; - /// Wheter base backup archive should be unlocked using the same password as the incremental archive + /// Whether base backup archive should be unlocked using the same password as the incremental archive bool use_same_password_for_base_backup = false; /// If it's true RESTORE won't stop on broken parts while restoring, instead they will be restored as detached parts From 12fb08648670dcf39659596ccb552f3462004fd8 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 16:25:07 +0200 Subject: [PATCH 043/496] added test for use_same_password_for_base_backup --- ...se_same_password_for_base_backup.reference | 21 ++++++++ ...ackup_use_same_password_for_base_backup.sh | 50 +++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference create mode 100755 tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference new file mode 100644 index 00000000000..7354d50a7c0 --- /dev/null +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -0,0 +1,21 @@ +use_same_password_for_base_backup +base +BACKUP_CREATED +add_more_data_1 +inc_1 +BACKUP_CREATED +add_more_data_2 +inc_2 +BACKUP_CREATED +inc_2_bad +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +restore_inc_1 +RESTORED +restore_inc_2 +RESTORED +restore_inc_2_bad +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +count_inc_1 +20 +count_inc_2 +30 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh new file mode 100755 index 00000000000..a2b1a953e24 --- /dev/null +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS data; + DROP TABLE IF EXISTS data_1; + DROP TABLE IF EXISTS data_2; + CREATE TABLE data (key Int) ENGINE=MergeTree() ORDER BY tuple(); + INSERT INTO data SELECT * from numbers(10); +" + +echo 'use_same_password_for_base_backup' +echo "base" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 + +echo 'add_more_data_1' +$CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" + +echo "inc_1" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo 'add_more_data_2' +$CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" + +echo "inc_2" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "inc_2_bad" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" + +echo "restore_inc_1" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "restore_inc_2" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "restore_inc_2_bad" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" + +echo "count_inc_1" +$CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 + +echo "count_inc_2" +$CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_2" | cut -f2 + +exit 0 From 6f25aacd71948b682630889e75774eb4f2469aaa Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Tue, 9 Jul 2024 16:36:59 +0200 Subject: [PATCH 044/496] use CLICKHOUSE_TEST_NAME instead of CLICKHOUSE_TEST_UNIQUE_NAME for backup filename --- ...kup_use_same_password_for_base_backup.reference | 4 ++-- ...843_backup_use_same_password_for_base_backup.sh | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..1a331cca46b 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index a2b1a953e24..4c5bec3775c 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -15,31 +15,31 @@ $CLICKHOUSE_CLIENT -nm -q " echo 'use_same_password_for_base_backup' echo "base" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip') SETTINGS password='password';" | cut -f2 echo 'add_more_data_1' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" echo "inc_1" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo 'add_more_data_2' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" echo "inc_2" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From cdd955f421e442cdb776e95e8e7ae8608bc8636e Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 09:20:39 +0200 Subject: [PATCH 045/496] Revert "use CLICKHOUSE_TEST_NAME instead of CLICKHOUSE_TEST_UNIQUE_NAME for backup filename" This reverts commit 72f6368a2ede6a01390db770f1b9ddfa00d3f1fe. --- ...kup_use_same_password_for_base_backup.reference | 4 ++-- ...843_backup_use_same_password_for_base_backup.sh | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 1a331cca46b..7354d50a7c0 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index 4c5bec3775c..a2b1a953e24 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -15,31 +15,31 @@ $CLICKHOUSE_CLIENT -nm -q " echo 'use_same_password_for_base_backup' echo "base" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip') SETTINGS password='password';" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 echo 'add_more_data_1' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" echo "inc_1" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo 'add_more_data_2' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" echo "inc_2" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From ba1c6fe3ef3a7f263939b2ae9e6c249d9c6b38c0 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 09:23:19 +0200 Subject: [PATCH 046/496] shorter grep output --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- .../02843_backup_use_same_password_for_base_backup.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..cbcb6b4cb7c 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index a2b1a953e24..f2f1265c1a0 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -30,7 +30,7 @@ echo "inc_2" $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 @@ -39,7 +39,7 @@ echo "restore_inc_2" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From 01c8faf190462d508d9cb1b7d342d6ccfbeeff88 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 11:15:33 +0200 Subject: [PATCH 047/496] Revert "shorter grep output" This reverts commit b0cbf1495dd0bcaba828706e895347e4ec550e29. --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- .../02843_backup_use_same_password_for_base_backup.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index cbcb6b4cb7c..7354d50a7c0 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index f2f1265c1a0..a2b1a953e24 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -30,7 +30,7 @@ echo "inc_2" $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 @@ -39,7 +39,7 @@ echo "restore_inc_2" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From 9dae370569054b4933b374c12dee6461c938fea8 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 11:23:36 +0200 Subject: [PATCH 048/496] fix db name in backup_use_same_password_for_base_backup test reference file --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..cdcf0532cd9 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_default_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_default_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 From 523e0abb4ec329c0535602c43c17991f4ef043a3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Jul 2024 13:15:14 +0000 Subject: [PATCH 049/496] Remove debug logs --- src/Processors/Sources/RemoteSource.cpp | 7 +------ src/QueryPipeline/RemoteQueryExecutor.cpp | 10 +--------- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index f1d47f69782..e33613564a2 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -90,10 +90,7 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { if (async_immediate_work.exchange(false)) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work was true"); return; - } /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. @@ -114,13 +111,11 @@ void RemoteSource::asyncJobReady() if (!was_query_sent) return; - auto res = query_executor->readAsync(/*probe=*/true); + auto res = query_executor->readAsync(/*check_packet_type_only=*/true); if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is {}", async_immediate_work); work(); async_immediate_work = true; - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is true"); } } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 3ca05b53417..87f634b8334 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -486,10 +486,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "was_cancelled"); return ReadResult(Block()); - } if (has_postponed_packet) { @@ -520,17 +517,12 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet /// Check if packet is not ready yet. if (read_context->isInProgress()) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "read_context still in progress"); return ReadResult(read_context->getFileDescriptor()); - } - - const auto packet_type = read_context->getPacketType(); - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Packet type: {}", packet_type); if (check_packet_type_only) { has_postponed_packet = true; + const auto packet_type = read_context->getPacketType(); if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) { From b8fbfd227fb60e0f244bda716ef5a9bb89376986 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 15:41:08 +0800 Subject: [PATCH 050/496] format --- src/Interpreters/InterpreterInsertQuery.cpp | 1 - .../test.py | 40 ++++++++----------- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 4064cd82b67..181fb064b54 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -49,7 +49,6 @@ namespace ProfileEvents { extern const Event InsertQueriesWithSubqueries; extern const Event QueriesWithSubqueries; - extern const int QUERY_IS_PROHIBITED; } namespace DB diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index f098f130d2b..f25964d27b8 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -37,29 +37,25 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("INSERT INTO my_table VALUES (1, 'hello')") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "INSERT INTO my_table VALUES (1, 'hello')" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER TABLE my_table delete where 1") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER TABLE my_table delete where 1" + ) + + + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER table my_table update key = 1 where 1" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER table my_table update key = 1 where 1") - ) - - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER TABLE my_table drop partition 0") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER TABLE my_table drop partition 0" ) reading_node.query("SELECT * from my_table"); @@ -73,12 +69,10 @@ def test_disable_insertion_and_mutation(started_cluster): reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") - assert( - "new_column\tString" - in reading_node.query("DESC my_table") + assert "new_column\tString" in reading_node.query( + "DESC my_table" ) - assert( - "new_column\tString" - in writing_node.query("DESC my_table") + assert "new_column\tString" in writing_node.query( + "DESC my_table" ) From 2e1f679ceb05afe4d5d813eb4048555c6311c3e1 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 16 Jul 2024 18:35:33 +0200 Subject: [PATCH 051/496] add S3DiskNoKeyErrors metric --- src/Common/CurrentMetrics.cpp | 2 ++ src/IO/S3/Client.cpp | 21 +++++++++++++++------ 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 7c97e73f278..2fedba0175b 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -306,6 +306,8 @@ \ 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") \ + \ + M(S3DiskNoKeyErrors, "Number of no-key S3 disk errors") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 3b958dea046..db20420db9f 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -24,6 +24,7 @@ #include #include +#include #include #include @@ -43,6 +44,11 @@ namespace ProfileEvents extern const Event TinyS3Clients; } +namespace CurrentMetrics +{ + extern const Metric S3DiskNoKeyErrors; +} + namespace DB { @@ -379,10 +385,10 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - /// The next call is NOT a recurcive call - /// This is a virtuall call Aws::S3::S3Client::HeadObject(const Model::HeadObjectRequest&) - return enrichErrorMessage( - HeadObject(static_cast(request))); + if (isClientForDisk()) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + + return enrichErrorMessage(std::move(result)); } /// For each request, we wrap the request functions from Aws::S3::Client with doRequest @@ -402,8 +408,11 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { - return enrichErrorMessage( - doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); })); + auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); + if (!resp.IsSuccess() && isClientForDisk()) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + + return enrichErrorMessage(std::move(resp)); } Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const From 884dc496a0a978074d3e0bd70f4df8d0225e69c1 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 20:58:37 +0800 Subject: [PATCH 052/496] format --- .../test_disable_insertion_and_mutation/test.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index f25964d27b8..b6431690245 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -35,7 +35,9 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): - writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) + writing_node.query( + """CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """ + ) assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "INSERT INTO my_table VALUES (1, 'hello')" @@ -58,7 +60,7 @@ def test_disable_insertion_and_mutation(started_cluster): "ALTER TABLE my_table drop partition 0" ) - reading_node.query("SELECT * from my_table"); + reading_node.query("SELECT * from my_table") writing_node.query("INSERT INTO my_table VALUES (1, 'hello')") writing_node.query("ALTER TABLE my_table delete where 1") writing_node.query("ALTER table my_table update value = 'no hello' where 1") @@ -69,10 +71,6 @@ def test_disable_insertion_and_mutation(started_cluster): reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") - assert "new_column\tString" in reading_node.query( - "DESC my_table" - ) + assert "new_column\tString" in reading_node.query("DESC my_table") - assert "new_column\tString" in writing_node.query( - "DESC my_table" - ) + assert "new_column\tString" in writing_node.query("DESC my_table") From c01e2cbeea02ebecfc4dea4692baffff3087b043 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 22:58:30 +0800 Subject: [PATCH 053/496] format --- tests/integration/test_disable_insertion_and_mutation/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index b6431690245..b6da7ed548f 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -50,7 +50,6 @@ def test_disable_insertion_and_mutation(started_cluster): assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "ALTER TABLE my_table delete where 1" ) - assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "ALTER table my_table update key = 1 where 1" From 7fc8ee726e3ef2dfb7d778fbb1a70fb147a33067 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 18 Jul 2024 12:32:16 +0200 Subject: [PATCH 054/496] add replication lag and recovery time metrics --- src/Databases/DatabaseReplicated.cpp | 44 +++++++++---------- src/Databases/DatabaseReplicated.h | 10 ++++- src/Databases/DatabaseReplicatedWorker.cpp | 4 ++ src/Databases/DatabaseReplicatedWorker.h | 4 ++ src/Storages/System/StorageSystemClusters.cpp | 33 +++++++++----- src/Storages/System/StorageSystemClusters.h | 2 +- .../test_recovery_time_metric/__init__.py | 0 .../configs/config.xml | 41 +++++++++++++++++ .../test_recovery_time_metric/test.py | 26 +++++++++++ 9 files changed, 129 insertions(+), 35 deletions(-) create mode 100644 tests/integration/test_recovery_time_metric/__init__.py create mode 100644 tests/integration/test_recovery_time_metric/configs/config.xml create mode 100644 tests/integration/test_recovery_time_metric/test.py diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7ce2859e962..b11b9382732 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -338,42 +338,40 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const return std::make_shared(getContext()->getSettingsRef(), shards, params); } -std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr & cluster_) const +ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const { - Strings paths; + ReplicasInfo res; + + auto zookeeper = getZooKeeper(); const auto & addresses_with_failover = cluster_->getShardsAddresses(); const auto & shards_info = cluster_->getShardsInfo(); - for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) - { - for (const auto & replica : addresses_with_failover[shard_index]) - { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - } - } try { - auto current_zookeeper = getZooKeeper(); - auto res = current_zookeeper->exists(paths); + UInt32 max_log_ptr = parse(zookeeper->get(zookeeper_path + "/max_log_ptr")); - std::vector statuses; - statuses.resize(paths.size()); - - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error == Coordination::Error::ZOK) - statuses[i] = 1; - - return statuses; - } - catch (...) + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); + UInt32 log_ptr = parse(zookeeper->get(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr")); + bool is_active = zookeeper->exists(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + res.push_back(ReplicaInfo{ + .is_active = is_active, + .replication_lag = max_log_ptr - log_ptr, + .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, + }); + } + } + return res; + } catch (...) { tryLogCurrentException(log); return {}; } } - void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config_ref) { const auto & config_prefix = fmt::format("named_collections.{}", collection_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index eab5b2ff931..db02b5ef30f 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -17,6 +17,14 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; +struct ReplicaInfo +{ + bool is_active; + UInt32 replication_lag; + UInt64 recovery_time; +}; +using ReplicasInfo = std::vector; + class DatabaseReplicated : public DatabaseAtomic { public: @@ -84,7 +92,7 @@ public: static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, bool throw_if_noop); - std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; + ReplicasInfo tryGetReplicasInfo(const ClusterPtr & cluster_) const; void renameDatabase(ContextPtr query_context, const String & new_name) override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 1ef88dc03bc..cea2d123f87 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,6 +32,8 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { + initialization_duration_timer.emplace(); + while (!stop_flag) { try @@ -69,6 +71,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; + initialization_duration_timer.reset(); return true; } catch (...) @@ -78,6 +81,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } + initialization_duration_timer.reset(); return false; } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 41edf2221b8..aea3b71173d 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -36,6 +36,8 @@ public: DatabaseReplicated * const database, bool committed = false); /// NOLINT UInt32 getLogPointer() const; + + UInt64 getCurrentInitializationDurationMs() const { return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; } private: bool initializeMainThread() override; void initializeReplication(); @@ -56,6 +58,8 @@ private: ZooKeeperPtr active_node_holder_zookeeper; /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; + + std::optional initialization_duration_timer; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 160c8d6270e..0da4bd70cbd 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -31,6 +31,8 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() {"database_shard_name", std::make_shared(), "The name of the `Replicated` database shard (for clusters that belong to a `Replicated` database)."}, {"database_replica_name", std::make_shared(), "The name of the `Replicated` database replica (for clusters that belong to a `Replicated` database)."}, {"is_active", std::make_shared(std::make_shared()), "The status of the Replicated database replica (for clusters that belong to a Replicated database): 1 means 'replica is online', 0 means 'replica is offline', NULL means 'unknown'."}, + {"replication_lag", std::make_shared(std::make_shared()), "The replication lag of the `Replicated` database replica (for clusters that belong to a Replicated database)."}, + {"recovery_time", std::make_shared(std::make_shared()), "The recovery time of the `Replicated` database replica (for clusters that belong to a Replicated database), in milliseconds."}, }; description.setAliases({ @@ -46,24 +48,23 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co writeCluster(res_columns, name_and_cluster, {}); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & name_and_database : databases) + for (const auto & [database_name, database] : databases) { - if (const auto * replicated = typeid_cast(name_and_database.second.get())) + if (const auto * replicated = typeid_cast(database.get())) { - if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, {database_name, database_cluster}, + replicated->tryGetReplicasInfo(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + database_name, database_cluster}, + replicated->tryGetReplicasInfo(database_cluster)); } } } void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const std::vector & is_active) + const ReplicasInfo & replicas_info) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; @@ -99,10 +100,22 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); res_columns[i++]->insert(address.database_shard_name); res_columns[i++]->insert(address.database_replica_name); - if (is_active.empty()) + if (replicas_info.empty()) + { res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + } else - res_columns[i++]->insert(is_active[replica_idx++]); + { + const auto & replica_info = replicas_info[replica_idx++]; + res_columns[i++]->insert(replica_info.is_active); + res_columns[i++]->insert(replica_info.replication_lag); + if (replica_info.recovery_time != 0) + res_columns[i++]->insert(replica_info.recovery_time); + else + res_columns[i++]->insertDefault(); + } } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 0f7c792261d..ead123aa79e 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -27,7 +27,7 @@ protected: using NameAndCluster = std::pair>; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); + static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const ReplicasInfo & replicas_info); }; } diff --git a/tests/integration/test_recovery_time_metric/__init__.py b/tests/integration/test_recovery_time_metric/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_recovery_time_metric/configs/config.xml b/tests/integration/test_recovery_time_metric/configs/config.xml new file mode 100644 index 00000000000..bad9b1fa9ea --- /dev/null +++ b/tests/integration/test_recovery_time_metric/configs/config.xml @@ -0,0 +1,41 @@ + + 9000 + + + + + + + + + default + + + + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + + diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py new file mode 100644 index 00000000000..9ceb0cce288 --- /dev/null +++ b/tests/integration/test_recovery_time_metric/test.py @@ -0,0 +1,26 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/config.xml"], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_recovery_time_metric(start_cluster): + node.query("CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');") + node.query("CREATE TABLE rdb.t (x UInt32) ENGINE = MergeTree ORDER BY x;") + node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) + node.restart_clickhouse() + assert ( + node.query("SELECT any(recovery_time) FROM system.clusters;") + != "0\n" + ) + From a6d4db342b2fc83e385d549ba5ce9ebf9e63064e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Jul 2024 16:45:40 +0000 Subject: [PATCH 055/496] Automatic style fix --- .../integration/test_recovery_time_metric/test.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 9ceb0cce288..90155f81ba2 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -2,7 +2,9 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/config.xml"], with_zookeeper=True) +node = cluster.add_instance( + "node", main_configs=["configs/config.xml"], with_zookeeper=True +) @pytest.fixture(scope="module") @@ -15,12 +17,10 @@ def start_cluster(): def test_recovery_time_metric(start_cluster): - node.query("CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');") + node.query( + "CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');" + ) node.query("CREATE TABLE rdb.t (x UInt32) ENGINE = MergeTree ORDER BY x;") node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) node.restart_clickhouse() - assert ( - node.query("SELECT any(recovery_time) FROM system.clusters;") - != "0\n" - ) - + assert node.query("SELECT any(recovery_time) FROM system.clusters;") != "0\n" From e806123856f5ded0f2e92f4f4b42c38132276c15 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 18 Jul 2024 20:30:56 +0000 Subject: [PATCH 056/496] Fix non x86 build --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 87f634b8334..d7edbc9ed35 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet_type_only) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] bool check_packet_type_only) { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) From 48e7708d7bcf575123ea20cee9455e0a4cf26791 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 19 Jul 2024 10:29:13 +0800 Subject: [PATCH 057/496] fix compile error --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Interpreters/InterpreterDeleteQuery.cpp | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 8 ++++---- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 398fe31f1a9..9b5b5dfc20a 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index b37ec4de4ab..291c8e19db0 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 181fb064b54..aef6c1249d5 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -411,10 +412,6 @@ std::pair, std::vector> InterpreterInsertQuery::buildP if (!running_group) running_group = std::make_shared(getContext()); - if (getContext()->getServerSettings().disable_insertion_and_mutation - && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) - throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); - std::vector sink_chains; std::vector presink_chains; @@ -737,6 +734,9 @@ BlockIO InterpreterInsertQuery::execute() const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); + if (getContext()->getServerSettings().disable_insertion_and_mutation + && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); From 4cb862432c50848e3406899f5c7079b4cf1d62a8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 09:34:20 +0000 Subject: [PATCH 058/496] Rename method --- src/Processors/IProcessor.h | 2 +- src/Processors/Sources/RemoteSource.cpp | 2 +- src/Processors/Sources/RemoteSource.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 358983a2179..0776921a814 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,7 +221,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } - virtual void asyncJobReady() {} + virtual void onAsyncJobReady() {} /** You must call this method if 'prepare' returned ExpandPipeline. * This method cannot access any port, but it can create new ports for current processor. diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3ec2356a121..587f6e2001b 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -104,7 +104,7 @@ void RemoteSource::work() ISource::work(); } -void RemoteSource::asyncJobReady() +void RemoteSource::onAsyncJobReady() { chassert(async_read); diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index fa04985f101..2247c781584 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -32,7 +32,7 @@ public: int schedule() override { return fd; } - void asyncJobReady() override; + void onAsyncJobReady() override; void setStorageLimits(const std::shared_ptr & storage_limits_) override; From 8349d260952a6daeb84c653c37ac000cf5302cfd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 11:25:34 +0000 Subject: [PATCH 059/496] Simplified implementation --- src/Processors/Sources/RemoteSource.cpp | 17 +++++------ src/Processors/Sources/RemoteSource.h | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 37 +++++++++++++++++++++++ src/QueryPipeline/RemoteQueryExecutor.h | 3 ++ 4 files changed, 49 insertions(+), 10 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 587f6e2001b..46c27676e12 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -89,9 +89,6 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { - if (async_immediate_work.exchange(false)) - return; - /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. /// See issue: https://github.com/ClickHouse/ClickHouse/issues/60844 @@ -101,6 +98,13 @@ void RemoteSource::work() executor_finished = true; return; } + + if (preprocessed_packet) + { + preprocessed_packet = false; + return; + } + ISource::work(); } @@ -111,12 +115,7 @@ void RemoteSource::onAsyncJobReady() if (!was_query_sent) return; - auto res = query_executor->readAsync(/*check_packet_type_only=*/true); - if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) - { - work(); - async_immediate_work = true; - } + preprocessed_packet = query_executor->processParallelReplicaPacketIfAny(); } std::optional RemoteSource::tryGenerate() diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 2247c781584..22d3921708b 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -54,7 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - std::atomic_bool async_immediate_work{false}; + bool preprocessed_packet = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index d7edbc9ed35..b15e31a120f 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -936,4 +936,41 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } +bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() +{ +#if defined(OS_LINUX) + if (!read_context || (resent_query && recreate_read_context)) + { + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + + read_context = std::make_unique(*this); + recreate_read_context = false; + } + + { + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + + chassert(!has_postponed_packet); + + read_context->resume(); + if (read_context->isInProgress()) // <- nothing to process + return false; + + const auto packet_type = read_context->getPacketType(); + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + processPacket(read_context->getPacket()); + return true; + } + + has_postponed_packet = true; + return false; + } +#endif +} + } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 6849c3e0a07..6f56df71f1d 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -222,6 +222,9 @@ public: bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } + /// return true if parallel replica packet was processed + bool processParallelReplicaPacketIfAny(); + private: RemoteQueryExecutor( const String & query_, From 4e3fdfc2d6482d42b8e152911e24ee38b1bafc89 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 19 Jul 2024 13:26:35 +0200 Subject: [PATCH 060/496] Save writer thread id for debugging --- src/Common/SharedMutex.cpp | 10 +++++++++- src/Common/SharedMutex.h | 2 ++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Common/SharedMutex.cpp b/src/Common/SharedMutex.cpp index 1df09ca998a..7b00ef0b28b 100644 --- a/src/Common/SharedMutex.cpp +++ b/src/Common/SharedMutex.cpp @@ -1,4 +1,5 @@ #include +#include #ifdef OS_LINUX /// Because of futex @@ -12,6 +13,7 @@ namespace DB SharedMutex::SharedMutex() : state(0) , waiters(0) + , writer_thread_id(0) {} void SharedMutex::lock() @@ -32,16 +34,22 @@ void SharedMutex::lock() value |= writers; while (value & readers) futexWaitLowerFetch(state, value); + + writer_thread_id.store(getThreadId()); } bool SharedMutex::try_lock() { UInt64 value = 0; - return state.compare_exchange_strong(value, writers); + bool success = state.compare_exchange_strong(value, writers); + if (success) + writer_thread_id.store(getThreadId()); + return success; } void SharedMutex::unlock() { + writer_thread_id.store(0); state.store(0); if (waiters) futexWakeUpperAll(state); diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index 9215ff62af3..a53e2984239 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -36,6 +36,8 @@ private: alignas(64) std::atomic state; std::atomic waiters; + /// Is set while the lock is held in exclusive mode only to facilitate debugging + std::atomic writer_thread_id; }; } From 55d1656f4d0da2f23b2df719dabeed7999645349 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 19 Jul 2024 13:27:41 +0200 Subject: [PATCH 061/496] Moving is not safe, prohibit it --- src/Common/SharedMutex.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index a53e2984239..c77c8765885 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -19,6 +19,8 @@ public: ~SharedMutex() = default; SharedMutex(const SharedMutex &) = delete; SharedMutex & operator=(const SharedMutex &) = delete; + SharedMutex(SharedMutex &&) = delete; + SharedMutex & operator=(SharedMutex &&) = delete; // Exclusive ownership void lock() TSA_ACQUIRE(); From 53ea5510143ded0862fd51922077a7cdc1344fe2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 11:30:55 +0000 Subject: [PATCH 062/496] Remove unused code --- src/QueryPipeline/RemoteQueryExecutor.cpp | 14 +------------- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b15e31a120f..b78c38a4134 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] bool check_packet_type_only) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) @@ -519,18 +519,6 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] if (read_context->isInProgress()) return ReadResult(read_context->getFileDescriptor()); - if (check_packet_type_only) - { - has_postponed_packet = true; - const auto packet_type = read_context->getPacketType(); - if (packet_type == Protocol::Server::MergeTreeReadTaskRequest - || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) - { - return ReadResult(ReadResult::Type::ParallelReplicasToken); - } - return ReadResult(ReadResult::Type::Nothing); - } - auto read_result = processPacket(read_context->getPacket()); if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) return read_result; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 6f56df71f1d..7289e2a2243 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -183,7 +183,7 @@ public: ReadResult read(); /// Async variant of read. Returns ready block or file descriptor which may be used for polling. - ReadResult readAsync(bool check_packet_type_only = false); + ReadResult readAsync(); /// Receive all remain packets and finish query. /// It should be cancelled after read returned empty block. From 3f66b39a18a7bf271a9a9f97dfc075866e2409eb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 14:30:27 +0200 Subject: [PATCH 063/496] test replication lag metric --- .../0_stateless/03206_replication_lag_metric.reference | 4 ++++ .../queries/0_stateless/03206_replication_lag_metric.sql | 9 +++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.reference create mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.sql diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.reference b/tests/queries/0_stateless/03206_replication_lag_metric.reference new file mode 100644 index 00000000000..02f4a7264b1 --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.reference @@ -0,0 +1,4 @@ +0 +2 +0 +2 diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql new file mode 100644 index 00000000000..6b86553fcaf --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -0,0 +1,9 @@ +CREATE DATABASE rdb1 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica1'); +CREATE DATABASE rdb2 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica2'); + +SET distributed_ddl_task_timeout = 0; +CREATE TABLE rdb1.t (id UInt32) ENGINE = ReplicatedMergeTree ORDER BY id; +SELECT replication_lag FROM system.clusters; + +DROP DATABASE rdb1; +DROP DATABASE rdb2; From 245626e5789064fda39ccc7288b83162284a3617 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 14:30:44 +0200 Subject: [PATCH 064/496] small fix --- src/Storages/System/StorageSystemClusters.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index ead123aa79e..f6e08734896 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -1,10 +1,10 @@ #pragma once +#include #include #include #include - namespace DB { From 79ef630d85cb445a743ee2d5950197709d75325f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 15:25:08 +0200 Subject: [PATCH 065/496] fix tests --- .../0_stateless/02117_show_create_table_system.reference | 2 ++ tests/queries/0_stateless/03206_replication_lag_metric.sql | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 8f62eda9233..28356632a66 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -52,6 +52,8 @@ CREATE TABLE system.clusters `database_shard_name` String, `database_replica_name` String, `is_active` Nullable(UInt8), + `replication_lag` Nullable(UInt32), + `recovery_time` Nullable(UInt64), `name` String ALIAS cluster ) ENGINE = SystemClusters diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql index 6b86553fcaf..998c332a11c 100644 --- a/tests/queries/0_stateless/03206_replication_lag_metric.sql +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -1,9 +1,11 @@ +-- Tags: no-parallel + CREATE DATABASE rdb1 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica1'); CREATE DATABASE rdb2 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica2'); SET distributed_ddl_task_timeout = 0; CREATE TABLE rdb1.t (id UInt32) ENGINE = ReplicatedMergeTree ORDER BY id; -SELECT replication_lag FROM system.clusters; +SELECT replication_lag FROM system.clusters WHERE cluster IN ('rdb1', 'rdb2') ORDER BY cluster ASC, replica_num ASC; DROP DATABASE rdb1; DROP DATABASE rdb2; From f3fb729f53860d55db1d72ccfc88f9c5d018aea1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 20:12:14 +0000 Subject: [PATCH 066/496] Call onAsyncJobReady() --- src/Processors/Executors/ExecutorTasks.cpp | 2 ++ src/Processors/Executors/ExecutorTasks.h | 2 +- src/Processors/Executors/PipelineExecutor.h | 1 - 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/ExecutorTasks.cpp b/src/Processors/Executors/ExecutorTasks.cpp index 7e3bee239ef..d045f59a2e2 100644 --- a/src/Processors/Executors/ExecutorTasks.cpp +++ b/src/Processors/Executors/ExecutorTasks.cpp @@ -204,6 +204,8 @@ void ExecutorTasks::processAsyncTasks() while (auto task = async_task_queue.wait(lock)) { auto * node = static_cast(task.data); + node->processor->onAsyncJobReady(); + executor_contexts[task.thread_num]->pushAsyncTask(node); ++num_waiting_async_tasks; diff --git a/src/Processors/Executors/ExecutorTasks.h b/src/Processors/Executors/ExecutorTasks.h index 202ca253c6c..b2201873edf 100644 --- a/src/Processors/Executors/ExecutorTasks.h +++ b/src/Processors/Executors/ExecutorTasks.h @@ -28,7 +28,7 @@ class ExecutorTasks TaskQueue task_queue; /// Queue which stores tasks where processors returned Async status after prepare. - /// If multiple threads are using, main thread will wait for async tasks. + /// If multiple threads are used, main thread will wait for async tasks. /// For single thread, will wait for async tasks only when task_queue is empty. PollingQueue async_task_queue; diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 03f0f7f1a0a..ae119355cb5 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -9,7 +9,6 @@ #include #include -#include #include From 465a34d3dfe3e313471e10d59cab8219b3e5837e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 20:27:57 +0000 Subject: [PATCH 067/496] Simplify, fix build --- src/QueryPipeline/RemoteQueryExecutor.cpp | 48 +++++++++++------------ 1 file changed, 22 insertions(+), 26 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b78c38a4134..61a512bcfc5 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -924,41 +924,37 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } -bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() +bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() { #if defined(OS_LINUX) + + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + if (!read_context || (resent_query && recreate_read_context)) { - std::lock_guard lock(was_cancelled_mutex); - if (was_cancelled) - return false; - read_context = std::make_unique(*this); recreate_read_context = false; } - { - std::lock_guard lock(was_cancelled_mutex); - if (was_cancelled) - return false; + chassert(!has_postponed_packet); - chassert(!has_postponed_packet); - - read_context->resume(); - if (read_context->isInProgress()) // <- nothing to process - return false; - - const auto packet_type = read_context->getPacketType(); - if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) - { - processPacket(read_context->getPacket()); - return true; - } - - has_postponed_packet = true; + read_context->resume(); + if (read_context->isInProgress()) // <- nothing to process return false; - } -#endif -} + const auto packet_type = read_context->getPacketType(); + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + processPacket(read_context->getPacket()); + return true; + } + + has_postponed_packet = true; + +#endif + + return false; +} } From 8d7471f8bd2e0c6dc242231c4358448787e6c56f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 20 Jul 2024 00:03:40 +0200 Subject: [PATCH 068/496] Fix for deadlock in getDDLWorker --- src/Interpreters/Context.cpp | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 94bcb88ed53..48878733a00 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3490,18 +3490,22 @@ DDLWorker & Context::getDDLWorker() const if (shared->ddl_worker_startup_task) waitLoad(shared->ddl_worker_startup_task); // Just wait and do not prioritize, because it depends on all load and startup tasks - SharedLockGuard lock(shared->mutex); - if (!shared->ddl_worker) { - if (!hasZooKeeper()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); - - if (!hasDistributedDDL()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no DistributedDDL configuration in server config"); - - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "DDL background thread is not initialized"); + /// Only acquire the lock for reading ddl_worker field. + /// hasZooKeeper() and hasDistributedDDL() acquire the same lock as well and double acquisition of the lock in shared mode can lead + /// to a deadlock if an exclusive lock attempt is made in the meantime by another thread. + SharedLockGuard lock(shared->mutex); + if (shared->ddl_worker) + return *shared->ddl_worker; } - return *shared->ddl_worker; + + if (!hasZooKeeper()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); + + if (!hasDistributedDDL()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no DistributedDDL configuration in server config"); + + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "DDL background thread is not initialized"); } zkutil::ZooKeeperPtr Context::getZooKeeper() const From c2ac13291f3bf201f7189bd36f2c9be7c06aa886 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 09:06:13 +0100 Subject: [PATCH 069/496] fix tests --- tests/integration/test_recovery_time_metric/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 90155f81ba2..e4a44103b76 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -3,7 +3,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/config.xml"], with_zookeeper=True + "node", main_configs=["configs/config.xml"], with_zookeeper=True, stay_alive=True, ) From d040e436f3a1f8594070b04ec10cbf7391f6994a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Jul 2024 08:18:58 +0000 Subject: [PATCH 070/496] Automatic style fix --- tests/integration/test_recovery_time_metric/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index e4a44103b76..4dad844b950 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -3,7 +3,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/config.xml"], with_zookeeper=True, stay_alive=True, + "node", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, ) From bb0b29f6e50e74098fcc8a9b83150998f1bc2601 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 22 Jul 2024 12:11:56 +0200 Subject: [PATCH 071/496] Set writer_thread_id earlier, when new exclusive owener is waiting for existing readers to finish --- src/Common/SharedMutex.cpp | 6 ++++-- src/Common/SharedMutex.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/SharedMutex.cpp b/src/Common/SharedMutex.cpp index 7b00ef0b28b..2d63f8542b0 100644 --- a/src/Common/SharedMutex.cpp +++ b/src/Common/SharedMutex.cpp @@ -31,11 +31,13 @@ void SharedMutex::lock() break; } + /// The first step of acquiring the exclusive ownership is finished. + /// Now we just wait until all readers release the shared ownership. + writer_thread_id.store(getThreadId()); + value |= writers; while (value & readers) futexWaitLowerFetch(state, value); - - writer_thread_id.store(getThreadId()); } bool SharedMutex::try_lock() diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index c77c8765885..d2947645eca 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -38,7 +38,7 @@ private: alignas(64) std::atomic state; std::atomic waiters; - /// Is set while the lock is held in exclusive mode only to facilitate debugging + /// Is set while the lock is held (or is in the process of being acquired) in exclusive mode only to facilitate debugging std::atomic writer_thread_id; }; From 2dc264928f311e2f4d10001044d070b6a6a05471 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 11:33:51 +0000 Subject: [PATCH 072/496] Added tests, rewritten logic which engines and table functions to allow, added replace for create table ... AS table_function() syntax. --- docs/en/operations/settings/settings.md | 12 ++ src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.cpp | 3 +- src/Interpreters/InterpreterCreateQuery.cpp | 50 +++--- .../test_restore_external_engines/__init__.py | 0 .../configs/backups_disk.xml | 14 ++ .../configs/remote_servers.xml | 21 +++ .../test_restore_external_engines/test.py | 143 ++++++++++++++++++ 8 files changed, 217 insertions(+), 29 deletions(-) create mode 100644 tests/integration/test_restore_external_engines/__init__.py create mode 100644 tests/integration/test_restore_external_engines/configs/backups_disk.xml create mode 100644 tests/integration/test_restore_external_engines/configs/remote_servers.xml create mode 100644 tests/integration/test_restore_external_engines/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..65b8df7a9e2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,15 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## restore_replace_external_engines_to_null + +For testing purposes. Replaces all external engines to Null to not initiate external connections. + +Default value: `False` + +## restore_replace_external_table_functions_to_null + +For testing purposes. Replaces all external engines to Null to not initiate external connections. + +Default value: `False` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7bf97896357..e6d2cac359b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,7 +891,8 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ - M(Bool, restore_replace_external_engine_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ + M(Bool, restore_replace_external_engines_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ + M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a23d9d17da2..0abcfb0cfb9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,7 +79,8 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } + } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { + if (create.as_table_function) + { + if (getContext()->getSettingsRef().restore_replace_external_table_functions_to_null) + { + const auto & factory = TableFunctionFactory::instance(); + + auto properties = factory.tryGetProperties(create.as_table_function->as()->name); + if (properties && properties->allow_readonly) + return; + if (!create.storage) + { + auto storage_ast = std::make_shared(); + create.set(create.storage, storage_ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage should not be created yet, it's a bug."); + create.as_table_function = nullptr; + setNullTableEngine(*create.storage); + } return; + } if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) return; @@ -1010,34 +1031,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else if (create.storage->engine->name == "AzureBlobStorage" || - create.storage->engine->name == "AzureQueue" || - create.storage->engine->name == "COSN" || - create.storage->engine->name == "DeltaLake" || - create.storage->engine->name == "Dictionary" || - create.storage->engine->name == "Executable" || - create.storage->engine->name == "ExecutablePool" || - create.storage->engine->name == "ExternalDistributed" || - create.storage->engine->name == "File" || - create.storage->engine->name == "Hudi" || - create.storage->engine->name == "Iceberg" || - create.storage->engine->name == "JDBC" || - create.storage->engine->name == "Kafka" || - create.storage->engine->name == "MaterializedPostgreSQL" || - create.storage->engine->name == "MongoDB" || - create.storage->engine->name == "MySQL" || - create.storage->engine->name == "NATS" || - create.storage->engine->name == "ODBC" || - create.storage->engine->name == "OSS" || - create.storage->engine->name == "PostgreSQL" || - create.storage->engine->name == "RabbitMQ" || - create.storage->engine->name == "Redis" || - create.storage->engine->name == "S3" || - create.storage->engine->name == "S3Queue" || - create.storage->engine->name == "TinyLog" || - create.storage->engine->name == "URL") + else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) { - if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) + if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) setNullTableEngine(*create.storage); } return; diff --git a/tests/integration/test_restore_external_engines/__init__.py b/tests/integration/test_restore_external_engines/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_restore_external_engines/configs/backups_disk.xml b/tests/integration/test_restore_external_engines/configs/backups_disk.xml new file mode 100644 index 00000000000..f7d666c6542 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/backups_disk.xml @@ -0,0 +1,14 @@ + + + + + local + /backups/ + + + + + backups + /backups/ + + diff --git a/tests/integration/test_restore_external_engines/configs/remote_servers.xml b/tests/integration/test_restore_external_engines/configs/remote_servers.xml new file mode 100644 index 00000000000..76ad3618339 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + replica3 + 9000 + + + + + diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py new file mode 100644 index 00000000000..cde4b0deb00 --- /dev/null +++ b/tests/integration/test_restore_external_engines/test.py @@ -0,0 +1,143 @@ +import pytest + +import pymysql.cursors +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] + +node1 = cluster.add_instance("replica1", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node2 = cluster.add_instance("replica2", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node3 = cluster.add_instance("replica3", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +nodes = [node1, node2, node3] + +backup_id_counter = 0 + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}/')" + +def cleanup_nodes(nodes, dbname): + for node in nodes: + node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") + +def fill_nodes(nodes, dbname): + cleanup_nodes(nodes, dbname) + for node in nodes: + node.query(f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')") + +def drop_mysql_table(conn, tableName): + with conn.cursor() as cursor: + cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") + +def get_mysql_conn(cluster): + conn = pymysql.connect( + user="root", password="clickhouse", host=cluster.mysql8_ip, port=cluster.mysql8_port + ) + return conn + +def fill_tables(cluster, dbname): + fill_nodes(nodes, dbname) + + conn = get_mysql_conn(cluster) + + with conn.cursor() as cursor: + cursor.execute( + "DROP DATABASE IF EXISTS clickhouse" + ) + cursor.execute( + "CREATE DATABASE clickhouse" + ) + cursor.execute( + "DROP TABLE IF EXISTS clickhouse.inference_table" + ) + cursor.execute( + "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" + ) + cursor.execute( + "INSERT INTO clickhouse.inference_table VALUES (100, X'9fad5e9eefdfb449')" + ) + conn.commit() + + parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" + + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" + ) + node1.query(f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})") + + node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") + node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") + + expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" + assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") == expected + assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") == expected + assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + +def test_restore_table(start_cluster): + fill_tables(cluster, "replicated") + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + node2.query(f"BACKUP DATABASE replicated TO {backup_name}") + + node2.query("DROP TABLE replicated.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" + + node3.query(f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true") + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine") == "1\t100\n" + assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function") == "1\t100\n" + assert node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + cleanup_nodes(nodes, "replicated") + + +def test_restore_table_null(start_cluster): + fill_tables(cluster, "replicated2") + + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") + + node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated2.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" + + node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engine_to_null=1, restore_replace_external_table_functions_to_null=1") + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" + assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function") == "0\t0\n" + assert node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" + assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'") == "Null\nNull\n" + assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'") == "MergeTree\n" + cleanup_nodes(nodes, "replicated2") From af4c2fa8a405c53d7de6d9ed41d63988caf22b04 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 12:27:24 +0000 Subject: [PATCH 073/496] fix --- tests/integration/test_restore_external_engines/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py index cde4b0deb00..be2cae334e2 100644 --- a/tests/integration/test_restore_external_engines/test.py +++ b/tests/integration/test_restore_external_engines/test.py @@ -132,7 +132,7 @@ def test_restore_table_null(start_cluster): assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engine_to_null=1, restore_replace_external_table_functions_to_null=1") + node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1") node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" From ae5eccbf20b7198d6a3cc908e0186a384aba038a Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 22 Jul 2024 13:39:48 +0000 Subject: [PATCH 074/496] just a commit to trigger CI --- .../test_grant_and_revoke/test_without_table_engine_grant.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py index 210bb8ec465..4a5dfb83f79 100644 --- a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py @@ -60,6 +60,7 @@ def test_table_engine_and_source_grant(): ) # expecting grant POSTGRES instead of grant PostgreSQL due to discrepancy between source access type and table engine + # similarily, other sources should also use their own defined name instead of the name of table engine assert "grant POSTGRES ON *.*" in instance.query_and_get_error( """ CREATE TABLE test.table1(a Integer) From 5fa2db8e4828b004ac10d625df62efcc8711dc98 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 16:59:17 +0100 Subject: [PATCH 075/496] fix 01293_show_clusters stateless test --- tests/queries/0_stateless/01293_show_clusters.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01293_show_clusters.reference b/tests/queries/0_stateless/01293_show_clusters.reference index e140f207022..9569fcf2e37 100644 --- a/tests/queries/0_stateless/01293_show_clusters.reference +++ b/tests/queries/0_stateless/01293_show_clusters.reference @@ -1,3 +1,3 @@ test_shard_localhost -test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default -test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default +test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default 0 NULL +test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default 0 NULL From 2a893ed8cd10e6e3b7506f43b644b5037f96c49a Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 17:03:41 +0100 Subject: [PATCH 076/496] fxs --- src/IO/S3/Client.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index db20420db9f..325c820f8bd 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -388,7 +388,8 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const if (isClientForDisk()) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - return enrichErrorMessage(std::move(result)); + return enrichErrorMessage( + HeadObject(static_cast(request))); } /// For each request, we wrap the request functions from Aws::S3::Client with doRequest From 71cdf82643fb17b5b68003df314c54ebbca0842f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 Jul 2024 18:41:30 +0000 Subject: [PATCH 077/496] Fix: reset is_async_state flag --- src/Processors/Sources/RemoteSource.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 46c27676e12..2f9a30296be 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -115,7 +115,10 @@ void RemoteSource::onAsyncJobReady() if (!was_query_sent) return; + chassert(!preprocessed_packet); preprocessed_packet = query_executor->processParallelReplicaPacketIfAny(); + if (preprocessed_packet) + is_async_state = false; } std::optional RemoteSource::tryGenerate() From 7f5c58f599d34f690c4a04e4223a2f86a433d0e9 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 20:58:56 +0100 Subject: [PATCH 078/496] fxs --- tests/queries/0_stateless/01293_show_clusters.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01293_show_clusters.reference b/tests/queries/0_stateless/01293_show_clusters.reference index 9569fcf2e37..e140f207022 100644 --- a/tests/queries/0_stateless/01293_show_clusters.reference +++ b/tests/queries/0_stateless/01293_show_clusters.reference @@ -1,3 +1,3 @@ test_shard_localhost -test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default 0 NULL -test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default 0 NULL +test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default +test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default From 72f4919fdad5217f48bd83e51ce2d1f3f083087b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mi=D1=81hael=20Stetsyuk?= <59827607+mstetsyuk@users.noreply.github.com> Date: Mon, 22 Jul 2024 21:33:47 +0100 Subject: [PATCH 079/496] Update src/Common/CurrentMetrics.cpp Co-authored-by: Sema Checherinda <104093494+CheSema@users.noreply.github.com> --- src/Common/CurrentMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2fedba0175b..39198147794 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -307,7 +307,7 @@ 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") \ \ - M(S3DiskNoKeyErrors, "Number of no-key S3 disk errors") \ + M(S3DiskNoKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) From 223eee3f46b07c38de3223fb56575f9ecbc5bea7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 07:57:03 +0000 Subject: [PATCH 080/496] Comment to new IProcessor method --- src/Processors/IProcessor.h | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 0776921a814..94e93595f4e 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,6 +221,21 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } + /* The method is called right after asynchronous job is done + * i.e. when file descriptor returned by schedule() is readable. + * The sequence of method calls: + * ... prepare() -> schedule() -> onAsyncJobReady() -> work() ... + * See also comment to schedule() method + * + * It allows doing some preprocessing immediately after asynchronous job is done. + * The implementation should return control quickly, to avoid blocking another asynchronous completed jobs + * created by the same pipeline. + * + * Example, scheduling tasks for remote workers (file descriptor in this case is a socket) + * When the remote worker asks for the next task, doing it in onAsyncJobReady() we can provide it immediately. + * Otherwise, the returning of the next task for the remote worker can be delayed by current work done in the pipeline + * i.e. processor->work(), which will create unnecessary latency in query processing by remote workers Not Committed Yet + */ virtual void onAsyncJobReady() {} /** You must call this method if 'prepare' returned ExpandPipeline. From 932033fca9bdacbfdb544fac5389e03fa7732eeb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 10:55:45 +0100 Subject: [PATCH 081/496] use atomic to avoid data race --- src/Databases/DatabaseReplicatedWorker.cpp | 12 +++++++++--- src/Databases/DatabaseReplicatedWorker.h | 5 +++-- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index cea2d123f87..a9a74c5f56a 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,7 +32,8 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { - initialization_duration_timer.emplace(); + initialization_duration_timer.restart(); + initializing.store(true, std::memory_order_release); while (!stop_flag) { @@ -71,7 +72,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - initialization_duration_timer.reset(); + initializing.store(false, std::memory_order_relaxed); return true; } catch (...) @@ -81,7 +82,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - initialization_duration_timer.reset(); + initializing.store(false, std::memory_order_relaxed); return false; } @@ -463,4 +464,9 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const return max_id.load(); } +UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const +{ + return initializing.load(std::memory_order_acquire) ? initialization_duration_timer.elapsedMilliseconds() : 0; +} + } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index aea3b71173d..3e5887be825 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -37,7 +37,7 @@ public: UInt32 getLogPointer() const; - UInt64 getCurrentInitializationDurationMs() const { return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; } + UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -59,7 +59,8 @@ private: /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; - std::optional initialization_duration_timer; + Stopwatch initialization_duration_timer; + std::atomic initializing = false; }; } From aaf603035e31874d6d5bcd024d0f4040715baa72 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 13:35:37 +0100 Subject: [PATCH 082/496] check error type --- src/IO/S3/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 325c820f8bd..7196dfa9bdc 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -385,7 +385,7 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - if (isClientForDisk()) + if (isClientForDisk() && error.GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); return enrichErrorMessage( @@ -410,7 +410,7 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); - if (!resp.IsSuccess() && isClientForDisk()) + if (!resp.IsSuccess() && isClientForDisk() && resp.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); return enrichErrorMessage(std::move(resp)); From d74dc587d7a183225b7cf0846b85e8213dcb7fc0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 13:06:58 +0000 Subject: [PATCH 083/496] Fix comment --- src/Processors/IProcessor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 94e93595f4e..4fd00d5e164 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -234,7 +234,7 @@ public: * Example, scheduling tasks for remote workers (file descriptor in this case is a socket) * When the remote worker asks for the next task, doing it in onAsyncJobReady() we can provide it immediately. * Otherwise, the returning of the next task for the remote worker can be delayed by current work done in the pipeline - * i.e. processor->work(), which will create unnecessary latency in query processing by remote workers Not Committed Yet + * (by other processors), which will create unnecessary latency in query processing by remote workers */ virtual void onAsyncJobReady() {} From b25cad23ed3b90dc8c0903710dba0714bac7219c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 11:42:28 +0000 Subject: [PATCH 084/496] Use unique names for tables and files --- .../integration/test_storage_s3_queue/test.py | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index bf3c28c5429..cf24e91f36b 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -7,6 +7,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance import json +from uuid import uuid4 AVAILABLE_MODES = ["unordered", "ordered"] @@ -822,7 +823,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] - table_name = f"max_set_age" + table_name = f"max_set_age_{uuid4().hex}" dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" @@ -847,11 +848,11 @@ def test_max_set_age(started_cluster): ) create_mv(node, table_name, dst_table_name) - total_values = generate_random_files( + _ = generate_random_files( started_cluster, files_path, files_to_generate, row_num=1 ) - expected_rows = 10 + expected_rows = files_to_generate node.wait_for_log_line("Checking node limits") node.wait_for_log_line("Node limits check finished") @@ -865,11 +866,11 @@ def test_max_set_age(started_cluster): time.sleep(1) assert expected_rows == get_count() - assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) time.sleep(max_age + 5) - expected_rows = 20 + expected_rows *= 2 for _ in range(20): if expected_rows == get_count(): @@ -877,7 +878,7 @@ def test_max_set_age(started_cluster): time.sleep(1) assert expected_rows == get_count() - assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) paths_count = [ int(x) @@ -885,7 +886,7 @@ def test_max_set_age(started_cluster): f"SELECT count() from {dst_table_name} GROUP BY _path" ).splitlines() ] - assert 10 == len(paths_count) + assert files_to_generate == len(paths_count) for path_count in paths_count: assert 2 == path_count @@ -901,7 +902,8 @@ def test_max_set_age(started_cluster): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - put_s3_file_content(started_cluster, f"{files_path}/fff.csv", values_csv) + file_with_error = f"fff_{uuid4().hex}.csv" + put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) for _ in range(30): if failed_count + 1 == int( @@ -920,16 +922,17 @@ def test_max_set_age(started_cluster): node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( - "SELECT exception FROM system.s3queue WHERE file_name ilike '%fff.csv'" + f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}'" ) + assert 1 == int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv'" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}'" ) ) assert 1 == int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv' AND notEmpty(exception)" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" ) ) @@ -943,11 +946,11 @@ def test_max_set_age(started_cluster): node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( - "SELECT exception FROM system.s3queue WHERE file_name ilike '%fff.csv' ORDER BY processing_end_time DESC LIMIT 1" + f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}' ORDER BY processing_end_time DESC LIMIT 1" ) assert 1 < int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv' AND notEmpty(exception)" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" ) ) From 91b7001df6e827f801bd792e7bd9d96cdd947946 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 12:08:31 +0000 Subject: [PATCH 085/496] Refactor test to improve it - Create wait_for_condition that checks greedily for a period of time - Remove redundant checks - Allow other tests running in parallel to have `ObjectStorageQueueFailedFiles` errors --- .../integration/test_storage_s3_queue/test.py | 56 ++++++------------- 1 file changed, 16 insertions(+), 40 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index cf24e91f36b..e178b3b6608 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -860,24 +860,21 @@ def test_max_set_age(started_cluster): def get_count(): return int(node.query(f"SELECT count() FROM {dst_table_name}")) - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) + def wait_for_condition(check_function, max_wait_time=30): + before = time.time() + while time.time() - before < max_wait_time: + if check_function(): + return + time.sleep(0.1) + assert False - assert expected_rows == get_count() + wait_for_condition(lambda: get_count() == expected_rows) assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) time.sleep(max_age + 5) expected_rows *= 2 - - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) - - assert expected_rows == get_count() + wait_for_condition(lambda: get_count() == expected_rows) assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) paths_count = [ @@ -890,11 +887,12 @@ def test_max_set_age(started_cluster): for path_count in paths_count: assert 2 == path_count - failed_count = int( - node.query( + def get_object_storage_failures(): + return int(node.query( "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ) + )) + + failed_count = get_object_storage_failures() values = [ ["failed", 1, 1], @@ -905,31 +903,13 @@ def test_max_set_age(started_cluster): file_with_error = f"fff_{uuid4().hex}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) - for _ in range(30): - if failed_count + 1 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ): - break - time.sleep(1) - - assert failed_count + 1 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ) + wait_for_condition(lambda: failed_count + 1 <= get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}'" ) - assert 1 == int( - node.query( - f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}'" - ) - ) assert 1 == int( node.query( f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" @@ -938,11 +918,7 @@ def test_max_set_age(started_cluster): time.sleep(max_age + 1) - assert failed_count + 2 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles'" - ) - ) + assert failed_count + 2 <= get_object_storage_failures() node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( From f03d4bb7d5d40203bba68c4f8958d584f27ae881 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 15:07:53 +0000 Subject: [PATCH 086/496] Format with black --- .../integration/test_storage_s3_queue/test.py | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index e178b3b6608..4348857acd3 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -823,7 +823,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] - table_name = f"max_set_age_{uuid4().hex}" + table_name = "max_set_age" dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" @@ -848,9 +848,7 @@ def test_max_set_age(started_cluster): ) create_mv(node, table_name, dst_table_name) - _ = generate_random_files( - started_cluster, files_path, files_to_generate, row_num=1 - ) + _ = generate_random_files(started_cluster, files_path, files_to_generate, row_num=1) expected_rows = files_to_generate @@ -869,13 +867,17 @@ def test_max_set_age(started_cluster): assert False wait_for_condition(lambda: get_count() == expected_rows) - assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int( + node.query(f"SELECT uniq(_path) from {dst_table_name}") + ) time.sleep(max_age + 5) expected_rows *= 2 wait_for_condition(lambda: get_count() == expected_rows) - assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int( + node.query(f"SELECT uniq(_path) from {dst_table_name}") + ) paths_count = [ int(x) @@ -888,9 +890,11 @@ def test_max_set_age(started_cluster): assert 2 == path_count def get_object_storage_failures(): - return int(node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - )) + return int( + node.query( + "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" + ) + ) failed_count = get_object_storage_failures() @@ -900,6 +904,8 @@ def test_max_set_age(started_cluster): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() + + # use a different filename for each test to allow running a bunch of them sequentially with --count file_with_error = f"fff_{uuid4().hex}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) From e4b50c18c2c1918905bf44a8e1183f0cddd5a811 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Jul 2024 22:26:46 +0200 Subject: [PATCH 087/496] getauxval: Avoid crash under sanitizer re-exec due to high ASLR entropy --- base/glibc-compatibility/musl/getauxval.c | 38 +++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index ea5cff9fc11..86f9a546ee4 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -75,6 +75,44 @@ unsigned long NO_SANITIZE_THREAD __getauxval_procfs(unsigned long type) } static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) { +#if defined(__x86_64__) && defined(__has_feature) +# if __has_feature(memory_sanitizer) || __has_feature(thread_sanitizer) + /// Sanitizers are not compatible with high ASLR entropy, which is the default on modern Linux distributions, and + /// to workaround this limitation, TSAN and MSAN (couldn't see other sanitizers doing the same), re-exec the binary + /// without ASLR (see https://github.com/llvm/llvm-project/commit/0784b1eefa36d4acbb0dacd2d18796e26313b6c5) + + /// The problem we face is that, in order to re-exec, the sanitizer wants to use the original pathname in the call + /// and to get its value it uses getauxval (https://github.com/llvm/llvm-project/blob/20eff684203287828d6722fc860b9d3621429542/compiler-rt/lib/sanitizer_common/sanitizer_linux_libcdep.cpp#L985-L988). + /// Since we provide getauxval ourselves (to minimize the version dependency on runtime glibc), we are the ones + // being called and we fail horribly: + /// + /// ==301455==ERROR: MemorySanitizer: SEGV on unknown address 0x2ffc6d721550 (pc 0x5622c1cc0073 bp 0x000000000003 sp 0x7ffc6d721530 T301455) + /// ==301455==The signal is caused by a WRITE memory access. + /// #0 0x5622c1cc0073 in __auxv_init_procfs ./ClickHouse/base/glibc-compatibility/musl/getauxval.c:129:5 + /// #1 0x5622c1cbffe9 in getauxval ./ClickHouse/base/glibc-compatibility/musl/getauxval.c:240:12 + /// #2 0x5622c0d7bfb4 in __sanitizer::ReExec() crtstuff.c + /// #3 0x5622c0df7bfc in __msan::InitShadowWithReExec(bool) crtstuff.c + /// #4 0x5622c0d95356 in __msan_init (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x256356) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + /// #5 0x5622c0dfe878 in msan.module_ctor main.cc + /// #6 0x5622c1cc156c in __libc_csu_init (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x118256c) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + /// #7 0x73dc05dd7ea3 in __libc_start_main /usr/src/debug/glibc/glibc/csu/../csu/libc-start.c:343:6 + /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + + /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as + /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliar vector. + /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very + /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise + // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). + + /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and it we used + /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and + /// unconditionally return "/proc/self/exe" without any preparation. Theoretically this should be fine in + /// our case, as we don't load any libraries. That's the theory at least. + if (type == AT_EXECFN) + return (unsigned long)"/proc/self/exe"; +# endif +#endif + // For debugging: // - od -t dL /proc/self/auxv // - LD_SHOW_AUX= ls From fda11dc62d81b717b9ab06c8adc8554c827764bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Jul 2024 22:51:26 +0200 Subject: [PATCH 088/496] Typo --- base/glibc-compatibility/musl/getauxval.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 86f9a546ee4..b5bd2f114c2 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -99,7 +99,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as - /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliar vector. + /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). From 7612060d232a24dbd721597c8e33cd1f556cddd6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 25 Jul 2024 06:40:51 +0000 Subject: [PATCH 089/496] allow only equal types in lagInFrame and leadInFrame --- src/Processors/Transforms/WindowTransform.cpp | 15 ++------------ .../03210_lag_lead_inframe_types.reference | 20 +++++++++++++++++++ .../03210_lag_lead_inframe_types.sql | 4 ++++ 3 files changed, 26 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03210_lag_lead_inframe_types.reference create mode 100644 tests/queries/0_stateless/03210_lag_lead_inframe_types.sql diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 86421adf4fb..06ae2bfb25e 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2385,22 +2385,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction return; } - const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); - if (!supertype) - { + if (!argument_types[0]->equals(*argument_types[2])) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "There is no supertype for the argument type '{}' and the default value type '{}'", + "Argument type '{}' and the default value type '{}' are different", argument_types[0]->getName(), argument_types[2]->getName()); - } - if (!argument_types[0]->equals(*supertype)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The supertype '{}' for the argument type '{}' and the default value type '{}' is not the same as the argument type", - supertype->getName(), - argument_types[0]->getName(), - argument_types[2]->getName()); - } if (argument_types.size() > 3) { diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference new file mode 100644 index 00000000000..cc3b9a096b9 --- /dev/null +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference @@ -0,0 +1,20 @@ +0 +1 +2 +2 +2 +2 +2 +2 +2 +2 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql new file mode 100644 index 00000000000..5466cfe0fad --- /dev/null +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql @@ -0,0 +1,4 @@ +SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); +SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); From c2f85c6fd062dde095ee34178450dc94c245e691 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:43:02 +0800 Subject: [PATCH 090/496] support map type as first argument type --- .../functions/tuple-map-functions.md | 2 +- src/Functions/map.cpp | 131 +++++++++++------- .../0_stateless/01651_map_functions.reference | 3 + .../0_stateless/01651_map_functions.sql | 9 +- 4 files changed, 89 insertions(+), 56 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 24b356eca87..ae23387f6e5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 66cd10a3f0b..5319390fb70 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -1,14 +1,17 @@ -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include @@ -178,22 +181,28 @@ public: getName(), arguments.size()); - /// The first argument should always be Array. - /// Because key type can not be nested type of Map, which is Tuple - DataTypePtr key_type; - if (const auto * keys_type = checkAndGetDataType(arguments[0].get())) - key_type = keys_type->getNestedType(); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); + auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + { + DataTypePtr nested; + if (const auto * array_type = checkAndGetDataType(type.get())) + nested = array_type->getNestedType(); + else if (const auto * map_type = checkAndGetDataType(type.get())) + nested = std::make_shared(map_type->getKeyValueTypes()); + else + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument types of function {} must be Array or Map, but {} is given", + getName(), + type->getName()); - DataTypePtr value_type; - if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) - value_type = value_array_type->getNestedType(); - else if (const auto * value_map_type = checkAndGetDataType(arguments[1].get())) - value_type = std::make_shared(value_map_type->getKeyValueTypes()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName()); + return nested; + }; + auto key_type = get_nested_type(arguments[0]); + auto value_type = get_nested_type(arguments[1]); + + /// Remove Nullable from key_type if needed for map key must not be Nullable + key_type = removeNullableOrLowCardinalityNullable(key_type); DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -201,44 +210,62 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - bool is_keys_const = isColumnConst(*arguments[0].column); - ColumnPtr holder_keys; - const ColumnArray * col_keys; - if (is_keys_const) + auto get_array_column = [this](const ColumnPtr & column) -> std::pair { - holder_keys = arguments[0].column->convertToFullColumnIfConst(); - col_keys = checkAndGetColumn(holder_keys.get()); - } - else + bool is_const = isColumnConst(*column); + ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; + + const ColumnArray * col_res = nullptr; + if (const auto * col_array = checkAndGetColumn(holder.get())) + col_res = col_array; + else if (const auto * col_map = checkAndGetColumn(holder.get())) + col_res = &col_map->getNestedColumn(); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument columns of function {} must be Array or Map, but {} is given", + getName(), + holder->getName()); + + return {col_res, holder}; + }; + + auto [col_keys, key_holder] = get_array_column(arguments[0].column); + + /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + ColumnPtr data_keys = col_keys->getDataPtr(); + if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - col_keys = checkAndGetColumn(arguments[0].column.get()); + std::cout << "data keys is nullable" << std::endl; + const NullMap * null_map = nullptr; + if (const auto * nullable = checkAndGetColumn(data_keys.get())) + { + null_map = &nullable->getNullMapData(); + data_keys = nullable->getNestedColumnPtr(); + } + else if (const auto * low_cardinality = checkAndGetColumn(data_keys.get())) + { + if (const auto * nullable_dict = checkAndGetColumn(low_cardinality->getDictionaryPtr().get())) + { + null_map = &nullable_dict->getNullMapData(); + data_keys = ColumnLowCardinality::create(nullable_dict->getNestedColumnPtr(), low_cardinality->getIndexesPtr()); + } + } + + if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); } - if (!col_keys) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName()); - - bool is_values_const = isColumnConst(*arguments[1].column); - ColumnPtr holder_values; - if (is_values_const) - holder_values = arguments[1].column->convertToFullColumnIfConst(); - else - holder_values = arguments[1].column; - - const ColumnArray * col_values; - if (const auto * col_values_array = checkAndGetColumn(holder_values.get())) - col_values = col_values_array; - else if (const auto * col_values_map = checkAndGetColumn(holder_values.get())) - col_values = &col_values_map->getNestedColumn(); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName()); - + auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); - const auto & data_keys = col_keys->getDataPtr(); const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets); + std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; + auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); + std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 471da5586b7..9114aa419b1 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -52,3 +52,6 @@ {1:4,2:5} {1:4,2:5} {1:4,2:5} +{1:3,2:4} +{1:3,2:4} +{1:3,2:4} {(1,3):'a',(2,4):'b'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index cf2460fce2c..4604ddd6db1 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -67,12 +67,15 @@ select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} +select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); -select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], [4, 5]); select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, 5])) from numbers(2); + +select mapFromArrays([1,2], [3,4]); +select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); +select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); From 65573871485c2e8ca45d791551856fd2f8622cf9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 11:05:36 +0200 Subject: [PATCH 091/496] Try calcualting memory with numactl if membind used --- .gitmodules | 3 + base/base/CMakeLists.txt | 4 ++ base/base/getMemoryAmount.cpp | 26 +++++++- contrib/CMakeLists.txt | 2 + contrib/numactl | 1 + contrib/numactl-cmake/CMakeLists.txt | 20 +++++++ contrib/numactl-cmake/include/config.h | 82 ++++++++++++++++++++++++++ programs/server/Server.cpp | 27 +++++++++ src/Common/config.h.in | 1 + src/configure_config.cmake | 3 + 10 files changed, 168 insertions(+), 1 deletion(-) create mode 160000 contrib/numactl create mode 100644 contrib/numactl-cmake/CMakeLists.txt create mode 100644 contrib/numactl-cmake/include/config.h diff --git a/.gitmodules b/.gitmodules index 12d865307d8..b5d7e1e56b3 100644 --- a/.gitmodules +++ b/.gitmodules @@ -372,3 +372,6 @@ [submodule "contrib/double-conversion"] path = contrib/double-conversion url = https://github.com/ClickHouse/double-conversion.git +[submodule "contrib/numactl"] + path = contrib/numactl + url = https://github.com/numactl/numactl.git diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 159502c9735..451a6eb5e8b 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -46,6 +46,10 @@ if (TARGET ch_contrib::crc32_s390x) target_link_libraries(common PUBLIC ch_contrib::crc32_s390x) endif() +if (TARGET ch_contrib::numactl) + target_link_libraries(common PUBLIC ch_contrib::numactl) +endif() + target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..") target_link_libraries (common diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index afdb6ba068a..b8162146496 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -4,12 +4,17 @@ #include #include -#include #include #include #include +#include "config.h" + +#if USE_NUMACTL +#include +#endif + namespace { @@ -63,6 +68,25 @@ uint64_t getMemoryAmountOrZero() uint64_t memory_amount = num_pages * page_size; +#if USE_NUMACTL + if (numa_available() != -1) + { + auto * membind = numa_get_membind(); + if (!numa_bitmask_equal(membind, numa_all_nodes_ptr)) + { + uint64_t total_numa_memory = 0; + auto max_node = numa_max_node(); + for (int i = 0; i <= max_node; ++i) + { + if (numa_bitmask_isbitset(membind, i)) + total_numa_memory += numa_node_size(i, nullptr); + } + + memory_amount = total_numa_memory; + } + } +#endif + /// Respect the memory limit set by cgroups v2. auto limit_v2 = getCgroupsV2MemoryLimit(); if (limit_v2.has_value() && *limit_v2 < memory_amount) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 90ae5981a21..977efda15ff 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -230,6 +230,8 @@ add_contrib (libssh-cmake libssh) add_contrib (prometheus-protobufs-cmake prometheus-protobufs prometheus-protobufs-gogo) +add_contrib(numactl-cmake numactl) + # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/numactl b/contrib/numactl new file mode 160000 index 00000000000..3871b1c42fc --- /dev/null +++ b/contrib/numactl @@ -0,0 +1 @@ +Subproject commit 3871b1c42fc71bceadafd745d2eff5dddfc2d67e diff --git a/contrib/numactl-cmake/CMakeLists.txt b/contrib/numactl-cmake/CMakeLists.txt new file mode 100644 index 00000000000..5d086366c7f --- /dev/null +++ b/contrib/numactl-cmake/CMakeLists.txt @@ -0,0 +1,20 @@ +option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_NUMACTL) + message (STATUS "Not using numactl") + return() +endif () + +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/numactl") + +set (SRCS + "${LIBRARY_DIR}/libnuma.c" + "${LIBRARY_DIR}/syscall.c" +) + +add_library(_numactl ${SRCS}) + +target_include_directories(_numactl SYSTEM PRIVATE include) +target_include_directories(_numactl SYSTEM PUBLIC "${LIBRARY_DIR}") + +add_library(ch_contrib::numactl ALIAS _numactl) diff --git a/contrib/numactl-cmake/include/config.h b/contrib/numactl-cmake/include/config.h new file mode 100644 index 00000000000..a304db38e53 --- /dev/null +++ b/contrib/numactl-cmake/include/config.h @@ -0,0 +1,82 @@ +/* config.h. Generated from config.h.in by configure. */ +/* config.h.in. Generated from configure.ac by autoheader. */ + +/* Checking for symver attribute */ +#define HAVE_ATTRIBUTE_SYMVER 0 + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDIO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to the sub-directory where libtool stores uninstalled libraries. */ +#define LT_OBJDIR ".libs/" + +/* Name of package */ +#define PACKAGE "numactl" + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "numactl" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "numactl 2.1" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "numactl" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "2.1" + +/* Define to 1 if all of the C89 standard headers exist (not just the ones + required in a freestanding environment). This macro is provided for + backward compatibility; new code need not use it. */ +#define STDC_HEADERS 1 + +/* If the compiler supports a TLS storage class define it to that here */ +#define TLS __thread + +/* Version number of package */ +#define VERSION "2.1" + +/* Number of bits in a file offset, on hosts where this is settable. */ +/* #undef _FILE_OFFSET_BITS */ + +/* Define to 1 on platforms where this makes off_t a 64-bit type. */ +/* #undef _LARGE_FILES */ + +/* Number of bits in time_t, on hosts where this is settable. */ +/* #undef _TIME_BITS */ + +/* Define to 1 on platforms where this makes time_t a 64-bit type. */ +/* #undef __MINGW_USE_VC2005_COMPAT */ diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..619a72ff200 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -140,6 +140,11 @@ # include #endif +#if USE_NUMACTL +#include +#endif + + #include /// A minimal file used when the server is run without installation INCBIN(resource_embedded_xml, SOURCE_DIR "/programs/server/embedded.xml"); @@ -754,6 +759,28 @@ try setenv("OPENSSL_CONF", config_dir.c_str(), true); /// NOLINT } +#if USE_NUMACTL + if (numa_available() != -1) + { + auto * membind = numa_get_membind(); + if (!numa_bitmask_equal(membind, numa_all_nodes_ptr)) + { + uint64_t total_numa_memory = 0; + auto max_node = numa_max_node(); + for (int i = 0; i <= max_node; ++i) + { + if (numa_bitmask_isbitset(membind, i)) + total_numa_memory += numa_node_size(i, nullptr); + } + + LOG_INFO( + log, + "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", + ReadableSize(total_numa_memory)); + } + } +#endif + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); diff --git a/src/Common/config.h.in b/src/Common/config.h.in index f68701d5d10..6a0090130a3 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -64,6 +64,7 @@ #cmakedefine01 USE_LIBARCHIVE #cmakedefine01 USE_POCKETFFT #cmakedefine01 USE_PROMETHEUS_PROTOBUFS +#cmakedefine01 USE_NUMACTL /// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. /// That's why we use absolute paths. diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 75f61baa854..d22bf674df4 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -173,5 +173,8 @@ endif() if (TARGET ch_contrib::prometheus_protobufs) set(USE_PROMETHEUS_PROTOBUFS 1) endif() +if (TARGET ch_contrib::numactl) + set(USE_NUMACTL 1) +endif() set(SOURCE_DIR ${PROJECT_SOURCE_DIR}) From e181ccd0173c46d31867097532f64df0be3944da Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:53:51 +0800 Subject: [PATCH 092/496] update doc --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index ae23387f6e5..db66188b1f5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** From a32c702caa142d15bc3e5bc51ca90240d5d010a9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 18:23:47 +0800 Subject: [PATCH 093/496] fix style --- src/Functions/map.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 5319390fb70..a8e5f7ad90e 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -236,7 +236,6 @@ public: ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - std::cout << "data keys is nullable" << std::endl; const NullMap * null_map = nullptr; if (const auto * nullable = checkAndGetColumn(data_keys.get())) { @@ -263,9 +262,7 @@ public: const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); - std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; From 7fedc0ffbee9d04e0352037021a127cea93cbbfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 12:26:37 +0200 Subject: [PATCH 094/496] Update base/glibc-compatibility/musl/getauxval.c Co-authored-by: Alexander Gololobov --- base/glibc-compatibility/musl/getauxval.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index b5bd2f114c2..28cb0f8d005 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -99,12 +99,12 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as - /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliary vector. + /// most global variables aren't initialized or available yet, so we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). - /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and it we used + /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and if we used /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and /// unconditionally return "/proc/self/exe" without any preparation. Theoretically this should be fine in /// our case, as we don't load any libraries. That's the theory at least. From beb506a5b8179f5c88a6f5fc90d62b8e74bf0d35 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 12:37:05 +0200 Subject: [PATCH 095/496] added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00307_format_xml.reference | 41 ++++++++++++++++++ .../queries/0_stateless/00307_format_xml.sql | 3 ++ .../0_stateless/00309_formats.reference | Bin 18537 -> 18736 bytes tests/queries/0_stateless/00309_formats.sql | 5 +++ 4 files changed, 49 insertions(+) diff --git a/tests/queries/0_stateless/00307_format_xml.reference b/tests/queries/0_stateless/00307_format_xml.reference index 2d9badc5a3e..14e74653d4f 100644 --- a/tests/queries/0_stateless/00307_format_xml.reference +++ b/tests/queries/0_stateless/00307_format_xml.reference @@ -1,3 +1,4 @@ +unnamed columns in tuple @@ -54,3 +55,43 @@ 1 +named columns in tuple + + + + + + s + String + + + time + DateTime + + + tpl + Tuple(String, DateTime) + + + + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + 1 + diff --git a/tests/queries/0_stateless/00307_format_xml.sql b/tests/queries/0_stateless/00307_format_xml.sql index 29c733bb186..22566112bc7 100644 --- a/tests/queries/0_stateless/00307_format_xml.sql +++ b/tests/queries/0_stateless/00307_format_xml.sql @@ -1,2 +1,5 @@ SET output_format_write_statistics = 0; +SELECT 'unnamed columns in tuple'; SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; +SELECT 'named columns in tuple'; +SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML;` diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index e637ee0363a7b35152a155ae3fa73a4f451d5148..a63720618ba54c6cc456f3356512449322dc2e80 100644 GIT binary patch delta 149 zcmaDkfpNnm#toSsmduQKj47-or3E>uY@wccC1xfpE{P?n;vparjXVX2n2th<0!Ubs ziva?zYfU!f5;0_8gmBMl=W(WRf=%Iq=->n?=fp5&@&sYg$;X8ig*A*!&2-d_3=9l( M4J>sHjny@|06s7$QUCw| delta 9 QcmdlmiSgwG#toSs02V|99{>OV diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index b0939c00a10..691fc6e7ab6 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -9,3 +9,8 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; + +SET enable_named_columns_in_function_tuple = 1; + +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes; From 2988e13050f0ab8a06e36ec8fe745386a214141b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 13:55:01 +0200 Subject: [PATCH 096/496] Free bitmask --- base/base/getMemoryAmount.cpp | 1 + programs/server/Server.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index b8162146496..56cddbfd628 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -84,6 +84,7 @@ uint64_t getMemoryAmountOrZero() memory_amount = total_numa_memory; } + numa_bitmask_free(membind); } #endif diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 619a72ff200..b9a7c298f00 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -778,6 +778,8 @@ try "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", ReadableSize(total_numa_memory)); } + + numa_bitmask_free(membind); } #endif From cd06945a03df0f8dbec6ff82332236caf86fbff3 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:05:25 +0000 Subject: [PATCH 097/496] Fix crash with Variant + AggregateFunction type --- src/Columns/ColumnAggregateFunction.cpp | 6 +- ...ant_with_aggregate_function_type.reference | 6 ++ ...0_variant_with_aggregate_function_type.sql | 60 +++++++++++++++++++ 3 files changed, 71 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference create mode 100644 tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index e26fe790a8e..9934970c868 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -330,7 +330,11 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_ void ColumnAggregateFunction::expand(const Filter & mask, bool inverted) { - expandDataByMask(data, mask, inverted); + ensureOwnership(); + Arena & arena = createOrGetArena(); + char * default_ptr = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); + func->create(default_ptr); + expandDataByMask(data, mask, inverted, default_ptr); } ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const diff --git a/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference new file mode 100644 index 00000000000..105e8e7d8bd --- /dev/null +++ b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference @@ -0,0 +1,6 @@ + 500 +fail 500 + 499 +fail 500 + 500 499 +fail 500 500 diff --git a/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql new file mode 100644 index 00000000000..cb9cdb0b456 --- /dev/null +++ b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql @@ -0,0 +1,60 @@ +SET allow_experimental_variant_type = 1; + +DROP TABLE IF EXISTS source; +CREATE TABLE source +( + Name String, + Value Int64 + +) ENGINE = MergeTree ORDER BY (); + +INSERT INTO source SELECT ['fail', 'success'][number % 2] as Name, number AS Value FROM numbers(1000); + +DROP TABLE IF EXISTS test_agg_variant; +CREATE TABLE test_agg_variant +( + Name String, + Value Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)) +) +ENGINE = MergeTree +ORDER BY (Name); + +INSERT INTO test_agg_variant +SELECT + Name, + t AS Value +FROM +( + SELECT + Name, + arrayJoin([ + uniqExactState(Value)::Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)), + avgState(Value)::Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)) + ]) AS t + FROM source + GROUP BY Name +); + +SELECT + Name, + uniqExactMerge(Value.`AggregateFunction(uniqExact, Int64)`) AS Value +FROM test_agg_variant +GROUP BY Name; + +SELECT + Name, + avgMerge(Value.`AggregateFunction(avg, Int64)`) AS Value +FROM test_agg_variant +GROUP BY Name; + +SELECT + Name, + uniqExactMerge(Value.`AggregateFunction(uniqExact, Int64)`) AS ValueUniq, + avgMerge(Value.`AggregateFunction(avg, Int64)`) AS ValueAvg +FROM test_agg_variant +GROUP BY Name; + + +DROP TABLE test_agg_variant; +DROP TABLE source; + From bd721950b0401f94be652c11015bd1985c283f3a Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 16:24:17 +0200 Subject: [PATCH 098/496] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 18736 -> 18666 bytes tests/queries/0_stateless/00309_formats.sql | 4 ++-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index a63720618ba54c6cc456f3356512449322dc2e80..f3ea45520bb50fb936caf6724e9fedf3cdd00b75 100644 GIT binary patch delta 54 zcmdlmiSgA$#tkJN8myt7c_kJsE{P?nVj-mkIjI_X3Sc1}g%kylsHVzCEg+hFz{3v! D3NRA? delta 99 zcmaDgk#WN$#tkJNdTgPdc_n5hEG~&9sp27}1v#l2c?u9Q9fcGHkgz5f0|Z>xnrz4= o!o|P{;hxo=JjcUt@=alWVGScwGaYp!0|NtH14~^)V|7g~0E0RhwEzGB diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index 691fc6e7ab6..0366cdeea5c 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -12,5 +12,5 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SET enable_named_columns_in_function_tuple = 1; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes; +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From 59f9c125044b6e56a3ded8034478eff79e930018 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 25 Jul 2024 14:37:47 +0000 Subject: [PATCH 099/496] Increase sleep time make sure there is a new failure The previous sleep was already adding +5s to make sure the TTL was properly applied, so we'd rather use the same value here instead of just 1s. --- tests/integration/test_storage_s3_queue/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 4348857acd3..2e339a9b5c9 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -871,7 +871,7 @@ def test_max_set_age(started_cluster): node.query(f"SELECT uniq(_path) from {dst_table_name}") ) - time.sleep(max_age + 5) + time.sleep(max_age + max_age / 2) expected_rows *= 2 wait_for_condition(lambda: get_count() == expected_rows) @@ -922,7 +922,7 @@ def test_max_set_age(started_cluster): ) ) - time.sleep(max_age + 1) + time.sleep(max_age + max_age / 2) assert failed_count + 2 <= get_object_storage_failures() From 5ea867231bafc01b4512989f351106b7afcc14af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 16:55:23 +0200 Subject: [PATCH 100/496] Read configuration for clickhouse-local from ~/.clickhouse-local Signed-off-by: Azat Khuzhin --- programs/local/LocalServer.cpp | 16 +++++++-- src/Common/Config/CMakeLists.txt | 1 + src/Common/Config/getLocalConfigPath.cpp | 46 ++++++++++++++++++++++++ src/Common/Config/getLocalConfigPath.h | 12 +++++++ 4 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 src/Common/Config/getLocalConfigPath.cpp create mode 100644 src/Common/Config/getLocalConfigPath.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 48e0cca7b73..ade4e0f49df 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -1,6 +1,7 @@ #include "LocalServer.h" #include +#include #include #include #include @@ -127,10 +128,21 @@ void LocalServer::initialize(Poco::Util::Application & self) { Poco::Util::Application::initialize(self); + const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe) + if (home_path_cstr) + home_path = home_path_cstr; + /// Load config files if exists - if (getClientConfiguration().has("config-file") || fs::exists("config.xml")) + std::string config_path; + if (getClientConfiguration().has("config-file")) + config_path = getClientConfiguration().getString("config-file"); + else if (config_path.empty() && fs::exists("config.xml")) + config_path = "config.xml"; + else if (config_path.empty()) + config_path = getLocalConfigPath(home_path).value_or(""); + + if (fs::exists(config_path)) { - const auto config_path = getClientConfiguration().getString("config-file", "config.xml"); ConfigProcessor config_processor(config_path, false, true); ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 09095ef5acc..2bd32b98bda 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -2,6 +2,7 @@ set (SRCS AbstractConfigurationComparison.cpp ConfigProcessor.cpp getClientConfigPath.cpp + getLocalConfigPath.cpp ConfigReloader.cpp YAMLParser.cpp ConfigHelper.cpp diff --git a/src/Common/Config/getLocalConfigPath.cpp b/src/Common/Config/getLocalConfigPath.cpp new file mode 100644 index 00000000000..afaa7f79026 --- /dev/null +++ b/src/Common/Config/getLocalConfigPath.cpp @@ -0,0 +1,46 @@ +#include + +#include +#include + + +namespace fs = std::filesystem; + +namespace DB +{ + +std::optional getLocalConfigPath(const std::string & home_path) +{ + std::string config_path; + bool found = false; + + std::vector names; + names.emplace_back("./clickhouse-local"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-local/config"); + names.emplace_back("/etc/clickhouse-local/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + + if (found) + return config_path; + + return std::nullopt; +} + +} diff --git a/src/Common/Config/getLocalConfigPath.h b/src/Common/Config/getLocalConfigPath.h new file mode 100644 index 00000000000..14625571d6c --- /dev/null +++ b/src/Common/Config/getLocalConfigPath.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Return path to existing configuration file. +std::optional getLocalConfigPath(const std::string & home_path); + +} From f2e83f092d1f677c4e0240e749f96766ff6e205c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 21:56:42 +0200 Subject: [PATCH 101/496] Patch getauxval for tsan re-exec --- base/glibc-compatibility/CMakeLists.txt | 10 ++++++++++ base/glibc-compatibility/musl/getauxval.c | 4 ++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index c967fa5b11b..8948e25cb8e 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -18,6 +18,16 @@ if (GLIBC_COMPATIBILITY) message (FATAL_ERROR "glibc_compatibility can only be used on x86_64 or aarch64.") endif () + if (SANITIZE STREQUAL thread) + # Disable TSAN instrumentation that conflicts with re-exec due to high ASLR entropy using getauxval + # See longer comment in __auxv_init_procfs + # In the case of tsan we need to make sure getauxval is not instrumented as that would introduce tsan + # internal calls to functions that depend on a state that isn't initialized yet + set_source_files_properties( + musl/getauxval.c + PROPERTIES COMPILE_FLAGS "-mllvm -tsan-instrument-func-entry-exit=false") + endif() + # Need to omit frame pointers to match the performance of glibc set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 28cb0f8d005..ec2cce1e4aa 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -102,7 +102,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// most global variables aren't initialized or available yet, so we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise - // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). + /// this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and if we used /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and @@ -237,7 +237,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_environ(unsigned long type) // - __auxv_init_procfs -> __auxv_init_environ -> __getauxval_environ static void * volatile getauxval_func = (void *)__auxv_init_procfs; -unsigned long getauxval(unsigned long type) +unsigned long NO_SANITIZE_THREAD getauxval(unsigned long type) { return ((unsigned long (*)(unsigned long))getauxval_func)(type); } From 3f70977cd660e4617d9bbd68cc229020adc57f98 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 25 Jul 2024 21:02:30 +0000 Subject: [PATCH 102/496] try to fix --- ...2572_query_views_log_background_thread.reference | 13 +++++++++---- .../02572_query_views_log_background_thread.sql | 8 ++++++-- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index 22dfaf93781..f867fd0d085 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -1,15 +1,14 @@ -- { echoOn } insert into buffer_02572 values (1); -- ensure that the flush was not direct +select * from buffer_02572; +1 select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; -select * from data_02572; -1 -select * from copy_02572; -1 +select sleepEachRow(1) from numbers(3*2) format Null; system flush logs; select count() > 0, lower(status::String), errorCodeToName(exception_code) from system.query_views_log where @@ -18,3 +17,9 @@ select count() > 0, lower(status::String), errorCodeToName(exception_code) group by 2, 3 ; 1 queryfinish OK +select * from buffer_02572; +1 +select * from data_02572; +1 +select * from copy_02572; +1 diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql index 939c189c5fe..2e9a62b71da 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql @@ -19,13 +19,13 @@ create materialized view mv_02572 to copy_02572 as select * from data_02572; -- { echoOn } insert into buffer_02572 values (1); -- ensure that the flush was not direct +select * from buffer_02572; select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; -select * from data_02572; -select * from copy_02572; +select sleepEachRow(1) from numbers(3*2) format Null; system flush logs; select count() > 0, lower(status::String), errorCodeToName(exception_code) @@ -34,3 +34,7 @@ select count() > 0, lower(status::String), errorCodeToName(exception_code) view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') group by 2, 3 ; + +select * from buffer_02572; +select * from data_02572; +select * from copy_02572; \ No newline at end of file From f0faa111d73c8512c1f88009f0ecfd1a804de45c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 Jul 2024 23:19:58 +0200 Subject: [PATCH 103/496] Fix wrong usage of input_format_max_bytes_to_read_for_schema_inference --- src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 1271cdfb7ad..e8956159714 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -253,7 +253,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation; format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; - format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; + format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_bytes_to_read_for_schema_inference; format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference; format_settings.schema_inference_hints = settings.schema_inference_hints; format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable; From 321766d0b8161a794f11835b4650d30b3723835b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 25 Jul 2024 22:51:14 +0000 Subject: [PATCH 104/496] Automatic style fix --- tests/performance/scripts/perf.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index f89784a0e0b..83d66997677 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -349,9 +349,7 @@ for query_index in queries_to_run: try: c.execute("SYSTEM JEMALLOC PURGE") - print( - f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}" - ) + print(f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}") except KeyboardInterrupt: raise except: From 287cce7d211b9386895a4fa898f87405eccb3e96 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 09:20:15 +0200 Subject: [PATCH 105/496] Fixes --- .gitmodules | 2 +- contrib/numactl | 2 +- docker/test/performance-comparison/run.sh | 1 + programs/server/Server.cpp | 9 ++++++++- 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index b5d7e1e56b3..7e0b4df4ad1 100644 --- a/.gitmodules +++ b/.gitmodules @@ -374,4 +374,4 @@ url = https://github.com/ClickHouse/double-conversion.git [submodule "contrib/numactl"] path = contrib/numactl - url = https://github.com/numactl/numactl.git + url = https://github.com/ClickHouse/numactl.git diff --git a/contrib/numactl b/contrib/numactl index 3871b1c42fc..8d13d63a05f 160000 --- a/contrib/numactl +++ b/contrib/numactl @@ -1 +1 @@ -Subproject commit 3871b1c42fc71bceadafd745d2eff5dddfc2d67e +Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 diff --git a/docker/test/performance-comparison/run.sh b/docker/test/performance-comparison/run.sh index 7afb5da59b1..6ef781fa4c8 100644 --- a/docker/test/performance-comparison/run.sh +++ b/docker/test/performance-comparison/run.sh @@ -13,6 +13,7 @@ entry="/usr/share/clickhouse-test/performance/scripts/entrypoint.sh" # https://www.kernel.org/doc/Documentation/filesystems/tmpfs.txt # Double-escaped backslashes are a tribute to the engineering wonder of docker -- # it gives '/bin/sh: 1: [bash,: not found' otherwise. +numactl --hardware node=$(( RANDOM % $(numactl --hardware | sed -n 's/^.*available:\(.*\)nodes.*$/\1/p') )); echo Will bind to NUMA node $node; numactl --cpunodebind=$node --membind=$node $entry diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b9a7c298f00..b818ff1f3a2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -775,9 +775,16 @@ try LOG_INFO( log, - "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", + "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes: {}", ReadableSize(total_numa_memory)); } + else + { + LOG_TRACE( + log, + "All NUMA nodes are used. Detected NUMA nodes: {}", + numa_num_configured_nodes()); + } numa_bitmask_free(membind); } From 10b36c09e9ffd96f53d2fdada725f9c999241dd1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 26 Jul 2024 15:38:26 +0800 Subject: [PATCH 106/496] support set orc reader time zone name --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/NativeORCBlockInputFormat.cpp | 6 +----- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 10 +++++----- 6 files changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a665326afc..a5220c3017c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1073,7 +1073,7 @@ class IColumn; M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \ - M(Bool, input_format_orc_read_use_writer_time_zone, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ + M(String, input_format_orc_reader_time_zone_name, "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index acd119c159b..457caa76bb6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -69,7 +69,7 @@ static std::initializer_listgetWriterTimezone(); - row_reader_options.setTimezoneName(writer_time_zone); - } + row_reader_options.setTimezoneName(format_settings.orc.reader_time_zone_name); row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); if (format_settings.orc.filter_push_down && sarg) { diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 27530c06237..7e931e16e48 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test SETTINGS session_timezone='Asia/Shanghai'" -$CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" +$CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From de84f4f045f5ece627ca8295a09a5f2cf1eab6aa Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 26 Jul 2024 08:13:21 +0000 Subject: [PATCH 107/496] add proper cast to lagInFrame/leadInFrame --- src/Processors/Transforms/WindowTransform.cpp | 86 ++++++++++++++++--- src/Processors/Transforms/WindowTransform.h | 1 + 2 files changed, 75 insertions(+), 12 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 06ae2bfb25e..006593edeaa 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,6 +16,9 @@ #include #include #include +#include +#include +#include #include #include @@ -75,6 +78,8 @@ public: virtual std::optional getDefaultFrame() const { return {}; } + virtual ColumnPtr castColumn(const Columns &, const std::vector &) { return nullptr; } + /// Is the frame type supported by this function. virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } }; @@ -1171,6 +1176,9 @@ void WindowTransform::appendChunk(Chunk & chunk) // Initialize output columns. for (auto & ws : workspaces) { + if (ws.window_function_impl) + block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices)); + block.output_columns.push_back(ws.aggregate_function->getResultType() ->createColumn()); block.output_columns.back()->reserve(block.rows); @@ -2358,6 +2366,8 @@ public: template struct WindowFunctionLagLeadInFrame final : public WindowFunction { + FunctionBasePtr func_cast = nullptr; + WindowFunctionLagLeadInFrame(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) @@ -2385,18 +2395,71 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction return; } - if (!argument_types[0]->equals(*argument_types[2])) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument type '{}' and the default value type '{}' are different", - argument_types[0]->getName(), - argument_types[2]->getName()); - if (argument_types.size() > 3) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function '{}' accepts at most 3 arguments, {} given", name, argument_types.size()); } + + if (argument_types[0]->equals(*argument_types[2])) + return; + + const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); + if (!supertype) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "There is no supertype for the argument type '{}' and the default value type '{}'", + argument_types[0]->getName(), + argument_types[2]->getName()); + } + if (!argument_types[0]->equals(*supertype)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The supertype '{}' for the argument type '{}' and the default value type '{}' is not the same as the argument type", + supertype->getName(), + argument_types[0]->getName(), + argument_types[2]->getName()); + } + + const auto from_name = argument_types[2]->getName(); + const auto to_name = argument_types[0]->getName(); + ColumnsWithTypeAndName arguments + { + { argument_types[2], "" }, + { + DataTypeString().createColumnConst(0, to_name), + std::make_shared(), + "" + } + }; + + auto get_cast_func = [&arguments] + { + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::accurate, {}); + return func_builder_cast->build(arguments); + }; + + func_cast = get_cast_func(); + + } + + ColumnPtr castColumn(const Columns & columns, const std::vector & idx) override + { + if (!func_cast) + return nullptr; + + ColumnsWithTypeAndName arguments + { + { columns[idx[2]], argument_types[2], "" }, + { + DataTypeString().createColumnConst(columns[idx[2]]->size(), argument_types[0]->getName()), + std::make_shared(), + "" + } + }; + + return func_cast->execute(arguments, argument_types[0], columns[idx[2]]->size()); } static DataTypePtr createResultType(const DataTypes & argument_types_, const std::string & name_) @@ -2446,12 +2509,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types.size() > 2) { // Column with default values is specified. - // The conversion through Field is inefficient, but we accept - // subtypes of the argument type as a default value (for convenience), - // and it's a pain to write conversion that respects ColumnNothing - // and ColumnConst and so on. - const IColumn & default_column = *current_block.input_columns[ - workspace.argument_column_indices[2]].get(); + const IColumn & default_column = + current_block.casted_columns[function_index] ? + *current_block.casted_columns[function_index].get() : + *current_block.input_columns[workspace.argument_column_indices[2]].get(); + to.insert(default_column[transform->current_row.row]); } else diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index 43fa6b28019..fe4f79e997c 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -50,6 +50,7 @@ struct WindowTransformBlock { Columns original_input_columns; Columns input_columns; + Columns casted_columns; MutableColumns output_columns; size_t rows = 0; From 5bf89a433128985944cb5dd6ad6ef40a9658ff52 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 08:42:52 +0000 Subject: [PATCH 108/496] Set a different instance dir when using pytest-xdist This allows executing in integration tests in parallel without directory clashes. --- tests/integration/helpers/cluster.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 548b58a17e8..0c8278048bf 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -434,6 +434,11 @@ class ClickHouseCluster: # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r"[^a-z0-9]", "", project_name.lower()) self.instances_dir_name = get_instances_dir(self.name) + xdist_worker = os.getenv("PYTEST_XDIST_WORKER") + if xdist_worker: + self.project_name += f"_{xdist_worker}" + self.instances_dir_name += f"_{xdist_worker}" + self.instances_dir = p.join(self.base_dir, self.instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, "docker.log") self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) From 2519f9ed4252020c6a9fb21ef1410c87f4053200 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 12:08:16 +0200 Subject: [PATCH 109/496] Only support archs --- contrib/numactl-cmake/CMakeLists.txt | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/contrib/numactl-cmake/CMakeLists.txt b/contrib/numactl-cmake/CMakeLists.txt index 5d086366c7f..a72ff11e485 100644 --- a/contrib/numactl-cmake/CMakeLists.txt +++ b/contrib/numactl-cmake/CMakeLists.txt @@ -1,4 +1,14 @@ -option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) +if (NOT ( + OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_LOONGARCH64)) +) + if (ENABLE_NUMACTL) + message (${RECONFIGURE_MESSAGE_LEVEL} + "numactl is disabled implicitly because the OS or architecture is not supported. Use -DENABLE_NUMACTL=0") + endif () + set (ENABLE_NUMACTL OFF) +else() + option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) +endif() if (NOT ENABLE_NUMACTL) message (STATUS "Not using numactl") From 72ebff825c0752e66cefa4f367ce43ff23d77703 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 Jul 2024 12:19:09 +0200 Subject: [PATCH 110/496] Reduce max time of 00763_long_lock_buffer_alter_destination_table --- ...ong_lock_buffer_alter_destination_table.sh | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 7e2384cfc52..c12b4426740 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -16,18 +16,39 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Bu ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" ${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" -function thread1() +function thread_alter() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 300 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + ALTER TABLE mt_00763_1 MODIFY column s UInt32; + ALTER TABLE mt_00763_1 MODIFY column s String; + " ||: + done } -function thread2() +function thread_query() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 2000 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + SELECT sum(length(s)) FROM buffer_00763_1; + " 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + done } -thread1 & -thread2 & +export -f thread_alter +export -f thread_query + +TIMEOUT=30 +thread_alter $TIMEOUT & +thread_query $TIMEOUT & wait From 98418120cd3167983b5436834d7c568cb42865af Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 10:21:43 +0000 Subject: [PATCH 111/496] Add parallel integration test execution to doc --- tests/integration/README.md | 67 ++++++++++++++++++++++++++++++++++--- 1 file changed, 62 insertions(+), 5 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index cde4cb05aec..5d4fa407e3f 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -16,7 +16,7 @@ Don't use Docker from your system repository. * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` * [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: -``` +```bash sudo -H pip install \ PyMySQL \ avro \ @@ -78,7 +78,7 @@ Notes: * Some tests maybe require a lot of resources (CPU, RAM, etc.). Better not try large tests like `test_distributed_ddl*` on your laptop. You can run tests via `./runner` script and pass pytest arguments as last arg: -``` +```bash $ ./runner --binary $HOME/ClickHouse/programs/clickhouse --odbc-bridge-binary $HOME/ClickHouse/programs/clickhouse-odbc-bridge --base-configs-dir $HOME/ClickHouse/programs/server/ 'test_ssl_cert_authentication -ss' Start tests ====================================================================================================== test session starts ====================================================================================================== @@ -102,7 +102,7 @@ test_ssl_cert_authentication/test.py::test_create_user PASSED ``` Path to binary and configs maybe specified via env variables: -``` +```bash $ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ $ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse $ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge @@ -121,6 +121,63 @@ test_odbc_interaction/test.py ...... [100%] You can just open shell inside a container by overwritting the command: ./runner --command=bash +### Parallel test execution + +On the CI, we run a number of parallel runners (5 at the time of this writing), each on its own +Docker container. These runner containers spawn more containers for the services needed such as +ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized +using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` +to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In other +words: tests are grouped by module for test functions and by class for test methods. This means that +any test within the same module (or any class) will never execute their tests in parallel. They'll +be executed on the same worker one after the other. + +If the test supports parallel and repeated execution, you can run a bunch of them in parallel to +look for flakiness. We use [pytest-repeat](https://pypi.org/project/pytest-repeat/) to set the +number of times we want to execute a test through the `--count` argument. Then, `-n` sets the number +of parallel workers for `pytest-xdist`. + +```bash +$ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ +$ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse +$ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge +$ ./runner 'test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' +Start tests +=============================================================================== test session starts ================================================================================ +platform linux -- Python 3.10.12, pytest-7.4.4, pluggy-1.5.0 -- /usr/bin/python3 +cachedir: .pytest_cache +rootdir: /ClickHouse/tests/integration +configfile: pytest.ini +plugins: reportlog-0.4.0, xdist-3.5.0, random-0.2, repeat-0.9.3, order-1.0.0, timeout-2.2.0 +timeout: 900.0s +timeout method: signal +timeout func_only: False +5 workers [10 items] +scheduling tests via LoadScheduling + +test_storage_s3_queue/test.py::test_max_set_age[9-10] +test_storage_s3_queue/test.py::test_max_set_age[7-10] +test_storage_s3_queue/test.py::test_max_set_age[5-10] +test_storage_s3_queue/test.py::test_max_set_age[1-10] +test_storage_s3_queue/test.py::test_max_set_age[3-10] +[gw3] [ 10%] PASSED test_storage_s3_queue/test.py::test_max_set_age[7-10] +test_storage_s3_queue/test.py::test_max_set_age[8-10] +[gw4] [ 20%] PASSED test_storage_s3_queue/test.py::test_max_set_age[9-10] +test_storage_s3_queue/test.py::test_max_set_age[10-10] +[gw0] [ 30%] PASSED test_storage_s3_queue/test.py::test_max_set_age[1-10] +test_storage_s3_queue/test.py::test_max_set_age[2-10] +[gw1] [ 40%] PASSED test_storage_s3_queue/test.py::test_max_set_age[3-10] +test_storage_s3_queue/test.py::test_max_set_age[4-10] +[gw2] [ 50%] PASSED test_storage_s3_queue/test.py::test_max_set_age[5-10] +test_storage_s3_queue/test.py::test_max_set_age[6-10] +[gw3] [ 60%] PASSED test_storage_s3_queue/test.py::test_max_set_age[8-10] +[gw4] [ 70%] PASSED test_storage_s3_queue/test.py::test_max_set_age[10-10] +[gw0] [ 80%] PASSED test_storage_s3_queue/test.py::test_max_set_age[2-10] +[gw1] [ 90%] PASSED test_storage_s3_queue/test.py::test_max_set_age[4-10] +[gw2] [100%] PASSED test_storage_s3_queue/test.py::test_max_set_age[6-10] +========================================================================== 10 passed in 120.65s (0:02:00) ========================================================================== +``` + ### Rebuilding the docker containers The main container used for integration tests lives in `docker/test/integration/base/Dockerfile`. Rebuild it with @@ -149,7 +206,7 @@ will automagically detect the types of variables and only the small diff of two If tests failing for mysterious reasons, this may help: -``` +```bash sudo service docker stop sudo bash -c 'rm -rf /var/lib/docker/*' sudo service docker start @@ -159,6 +216,6 @@ sudo service docker start On Ubuntu 20.10 and later in host network mode (default) one may encounter problem with nested containers not seeing each other. It happens because legacy and nftables rules are out of sync. Problem can be solved by: -``` +```bash sudo iptables -P FORWARD ACCEPT ``` From 60cca77c8a415142fe6f181b25aaed84232ea3c0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 26 Jul 2024 12:39:35 +0200 Subject: [PATCH 112/496] add example for materialize function --- .../functions/other-functions.md | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b7e4094f30e..797607e552a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -346,7 +346,9 @@ Result: ## materialize Turns a constant into a full column containing a single value. -Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. +Full columns and constants are represented differently in memory. +Functions usually execute different code for normal and constant arguments, although the result should typically be the same. +This function can be used to debug this behavior. **Syntax** @@ -354,6 +356,34 @@ Full columns and constants are represented differently in memory. Functions usua materialize(x) ``` +**Parameters** + +- `x` — A constant. [Constant](../functions/index.md/#constants). + +**Returned value** + +- A column containing a single value `x`. + +**Example** + +In the example below the `countMatches` function expects a constant second argument. +This behaviour can be debugged by using the `materialize` function to turn a constant into a full column, +verifying that the function throws an error for a non-constant argument. + +Query: + +```sql +SELECT countMatches('foobarfoo', 'foo'); +SELECT countMatches('foobarfoo', materialize('foo')); +``` + +Result: + +```response +2 +Code: 44. DB::Exception: Received from localhost:9000. DB::Exception: Illegal type of argument #2 'pattern' of function countMatches, expected constant String, got String +``` + ## ignore Accepts any arguments, including `NULL` and does nothing. Always returns 0. From 3f1dbdfce978bab2b2ce2aedecdbb5afbf54c4a0 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 11:01:10 +0000 Subject: [PATCH 113/496] Clarify documentation --- tests/integration/README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index 5d4fa407e3f..ab984b7bd04 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -127,10 +127,10 @@ On the CI, we run a number of parallel runners (5 at the time of this writing), Docker container. These runner containers spawn more containers for the services needed such as ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` -to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In other -words: tests are grouped by module for test functions and by class for test methods. This means that -any test within the same module (or any class) will never execute their tests in parallel. They'll -be executed on the same worker one after the other. +to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the +documentation words: this guarantees that all tests in a file run in the same worker. This means +that any test within the same file will never execute their tests in parallel. They'll be executed +on the same worker one after the other. If the test supports parallel and repeated execution, you can run a bunch of them in parallel to look for flakiness. We use [pytest-repeat](https://pypi.org/project/pytest-repeat/) to set the From 28e991708be1facd87c3760f7929cd5ddc299805 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 13:45:57 +0200 Subject: [PATCH 114/496] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 18666 -> 20353 bytes tests/queries/0_stateless/00309_formats.sql | 5 ----- 2 files changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index f3ea45520bb50fb936caf6724e9fedf3cdd00b75..cab311692be229716b58af39079275d3942b01cc 100644 GIT binary patch literal 20353 zcmeHP3wIP%5{6aK9zDL_&p18;axq=E-!KeHeM!>@7a6Ht;zJ#Jk0d+*gt^JbdYKDu4%*X9d$a^ZOYUl#FBHF2yr}q<;@67HiYtm^#Z|>iir*-% zDXuF{6fY}&tN5Mb_lj2(e^C5U@h8Qria#snd+xpO{s$g>=;22meeCfko}78A|MWA@J~z8>|MLgJ!58LIidkNiRoxsq zJUB8u`r@(UFP(V#6D$6wrgio0oJnBji=r;A-&v0K-- z%ZqMZ-|En<>suYVb$zQtx30$|*-G(uTzhnP-g&=^d;Asr39~@AuK#;~vDmn-A16x* zR`tnxBCGmdyM|AAwVs$&ebTP#ldV?ulqNAv(}O7;BueJfKp}=K;v>=KamanSU6UWrDN?_JXVk8gMI`72?`P<#P@i7&<`pS zWF+WF5R#xIK}v#_1ThI}667T4Nf4BvC_z$!rUX$5suE-+=t>Zlpe#XJg0=*43F;E$ zCFn~Kn4mC0VuHp5kqIgjWG3iL5SpMgL28251hENf6XYi7O%R-*I6-oP<^<6RsuN@< z=uQxxpgci(g7yUQ3F;H%C+JUb0Ko$U7Z7|va00;#1UC@;KyU=X69iWfd_iyq!5aj3 z5d48~#s?7ye?<5r!XFX-i10^*KO+1Q;g1M^M8t7~KO+1Q;g1M^MEE1Z9})hD9M2rk z64B_#G0j2EQO#k^am|4Ze?<5r!XFX-i0}tWF!-P!{1M@g2!BNQBf=jM{)q5Lgg+wu z5#f&re?<5r!XFX-i10^*KO+1Q;g1M^#G){NzzgHs7A`ETZCepq_933?i~U2rWWJXqI8KuOcyVRaKQz-H_k$cyu3VDiwye}g+f_bcJ1@D;TzuN5 zD@s+avnf(-W3}#V+EUgwouv-u47MnGiFGeI6M7J0X>pSftOBKaoh3fnm@GvczL`Ta z^L_N{6Fw0i&Ve~T1wJJ{6+SgSO&=q%!s~pgh07?nk!+)EGuh^{Eo58Dw&Hr*^k(>Z zybQ>^Hqtrc{32`%3@;;YAm1kG-$5QXD0j)XNqnD@?1}G`_*RMUmH1|f@0J>DqOcTU z*dnqtku4%yM7D@*5!oWLMaC8xTV$doTV!mJu|>uf8Czs*k+DV27CBqwY>|unY>~4? z&K5aasMw-ni;68Owy4;mVvC9`Dz>QDqGpSlEo!!?*`j8Pnk{O!sM(@si<&JOwrJR* zVT*<>8n$TIqG5}MEgH6H*a8(t*|H6)jlxDpGfP)fUuE8&1TrL@Sn5)r6Ya*MU7eL0wb!Wn76AR5ztX z#+6_}jZ<1=T!|M{I=Mw62Gvg4mvJR%Q1g@)8CT*4l}~AraV2z6|Kt`49@Ic(U&fXA zK_ygLWLya$)I+&Nf(SKH*_Ux8j!+qu78zGU3H4EFk#QxOP$lISi6>M_WnadXkV3sw zT4Y>_DpXCSg~!FbC|e#E)l=EBEh?z8Wm{BHWy|BDk}6xVZKpC4TZt>wRQasL73!+8 zFSZg_sIE$j*tQdl#8s5ILYJI}Nejl>nIuG~}N3YAyc7h8!d)L*4V z#+A526;@hgT!|~xV!1`)3iVjom;05tLRD5;5#VP`KA{>G~DrX+|J4NE4^tt78W7My1Ewbk!9(?&_}d~o*M=)&sS=v zm5YR#+gx;@nz8VMIrYYX7g?<00RC^)91khQIPJmG@OrgrFt^k;LYv_Lo5d@`!DxB3 zGDLsD_K3_;G)I{^%FR(>j!JV>nWNeqHPfRx<;2)BT_0Y1Xf&A9h!Y`)0|nvDH@x8n zHr{gEJg2=CS+6-2Y@=}a^V*~wFA#5d<+f6RtRKlbl5c8*WMo}Q)|X_RN!FWW-AUK{ zDc6tAH={hZdv7)Rc=}3mQ`+9<(5@$S>={hVObTHRbg2bmQ`U{6_yqMxorr-!m=tXtHQD> zEvwS9DlMzhvMMdB(y}TotJ1P6EvwS9sw}I@vZ^er%Cf2~tID#fEUU`0sw}I@vZ^hs z+On!GtJ<=vEvwqHsx7P9vZ^hs+OldatH!cwEUU({YAmb9vT7`=#}TLyo>A3TXJ|u*PXZY@b0cX?~)z)dEQU-ex~gv+M8t>k(P&*cKVkL~(9{aiZ0vklwdB?P?7?an*> zTu#9I-mbsX&!q*_8f<@;81VeWop<`V+<^DVU4N&aOAmPG-1T?*xeS5#(rtg2Bk;7w zop<`VG=X>8U4N&a%M^I;-Sv0+xnzOoK(@b27kFphop<`VjDh#+U4N&aOB#6h-u8EC z1J9b=d8ePt9C&Zv^>_NY_NY{DCT;>+ke)`2)`|ZGV?P@Y@r2-syKefBb6wVB0s=_`E`L+6m)Ezc_5xV{`a(e6A7cfwpXiGp`$XFl3xq_$>F>d{|a@uQ~ Lvprlx%YOd>&)TR8 delta 3052 zcmeIy$4}c(00wZIgtksdZ-w^I3JD}eL2?j6{R4XG1$t-?y;Q0c4wb5?2YL>Bj|8*# z-Xp>6y?59U_TGDMNaN%a?SDX3#j^bK`<|bk?ZfAlN^UDiNE{JVHgaD5e8+19DxExLIy^` zXcz-yVH}Ky2`~{R!DN^MQ}xOSA`L-0%z&9N3ueO{m<#h@J}iKRum~2z5?IQq+n-P( zP{A_D!g5#vD`6F^hBdGj*1>w%02^VGPHumqnY-wY76U{pY=iBv19rkL*bRGNFYJT; zPz?v*AoeGQ5Ddc+I10z$IGlhQI0>iVG@OC6a1PGH1;h6f#3F(vxC~d|DqMr>a070_ zEw~ML;4a*Q`|v>j|8;{y1V``~p1@Oh2G8LIyo6Wq8s5NLcn9yHg%&qjk~HD*O`kk{ zMhS#wBF4@26;n)ok$HM~`}q3#2LuKMhlGZON5~bCOjLACY+QUoVp4KSYFc_mW>$7i zZeD&tVNr2OsZv$OmRD3(RoB$k)i*RYHMg|3wRd!Ob@%l4^{WR4hlWQ+$Hpf#lT*_( zvvczci%ZKZt842Un_JsEyL2uP zs~KC(*lPbltFhlLRQ$(oSyIc$t;|{F>ph;XVgHfpC$+5H%8ajnp%+Y<&DY2OORU_k z(V7&oZmv$Gj7LgIiK)l0pFiA9KYx&VyyL(B=HYJsi3@C{mioxSQew%acGgGsx>HI? tDE$HfH|yd|adXt)D_oo(mI%Z& reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; - -SET enable_named_columns_in_function_tuple = 1; - -SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From d041df80aa112920f28d74ed26a0c8381808dafc Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 26 Jul 2024 12:14:26 +0000 Subject: [PATCH 115/496] Add test --- ..._to_read_for_schema_inference_in_cache.reference | 2 ++ ...x_bytes_to_read_for_schema_inference_in_cache.sh | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference create mode 100755 tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference new file mode 100644 index 00000000000..cd109daac52 --- /dev/null +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference @@ -0,0 +1,2 @@ +x Nullable(Int64) +schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh new file mode 100755 index 00000000000..8a77538f592 --- /dev/null +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS input_format_max_bytes_to_read_for_schema_inference=1000; +SELECT additional_format_info from system.schema_inference_cache" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.json + From aec7848525d2d6bd1cc9e7c573c25bd3b4ac79e7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 26 Jul 2024 13:04:44 +0000 Subject: [PATCH 116/496] fix --- src/Processors/Transforms/WindowTransform.cpp | 2 +- .../03210_lag_lead_inframe_types.reference | 20 +++++++++++++++++++ .../03210_lag_lead_inframe_types.sql | 4 ++-- 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 61be0c32a7d..1eac08780e9 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2424,7 +2424,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction argument_types[0]->getName(), argument_types[2]->getName()); } - + const auto from_name = argument_types[2]->getName(); const auto to_name = argument_types[0]->getName(); ColumnsWithTypeAndName arguments diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference index cc3b9a096b9..d4734a85e72 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference @@ -18,3 +18,23 @@ 7 8 9 +0 +1 +2 +2 +2 +2 +2 +2 +2 +2 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql index 5466cfe0fad..f6017ee6690 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql @@ -1,4 +1,4 @@ -SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } -SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); +SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); From 42384af0ef38dd326337e8cf18327871924f7359 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 25 Jul 2024 15:42:50 +0000 Subject: [PATCH 117/496] Fix crash when the connection is empty --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 6 ++++++ .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 3 +++ 2 files changed, 9 insertions(+) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index e1facec5b40..31779a32c1f 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -245,6 +245,9 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha connection = std::move(result.front().entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", files.size(), connection->getDescription(), @@ -303,6 +306,9 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett auto connection = std::move(result.front().entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + RemoteInserter remote(*connection, timeouts, distributed_header.insert_query, insert_settings, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..15998776d27 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -415,6 +415,9 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto result = pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); auto connection = std::move(result.front().entry); + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, connection->getDescription(), From 981135bfb104b5ecfa0f1da5533e3d12f6850838 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 13:38:42 +0000 Subject: [PATCH 118/496] Fix style check --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 5 +++-- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 ++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 31779a32c1f..5e7b4b979c7 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } /// Can the batch be split and send files from batch one-by-one instead? @@ -246,7 +247,7 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha compression_expected = connection->getCompression() == Protocol::Compression::Enable; if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", files.size(), @@ -307,7 +308,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); RemoteInserter remote(*connection, timeouts, distributed_header.insert_query, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 15998776d27..2bb0e720c72 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -416,7 +416,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto connection = std::move(result.front().entry); if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index b2ce62caf0a..f8bbc081e55 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -378,6 +378,8 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) auto results = shard_info.pool->getManyCheckedForInsert(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); job.connection_entry = std::move(results.front().entry); + if (job.connection_entry.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); } else { From ff5cd2051fc8bfd609a9040ffba02697283e69af Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 17:10:39 +0200 Subject: [PATCH 119/496] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- tests/queries/0_stateless/00307_format_xml.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00307_format_xml.sql b/tests/queries/0_stateless/00307_format_xml.sql index 22566112bc7..a7e0e628945 100644 --- a/tests/queries/0_stateless/00307_format_xml.sql +++ b/tests/queries/0_stateless/00307_format_xml.sql @@ -2,4 +2,4 @@ SET output_format_write_statistics = 0; SELECT 'unnamed columns in tuple'; SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; SELECT 'named columns in tuple'; -SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML;` +SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; From 343f1fa4bae219f7c287cb314ed6e04feb9a0de4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:42:06 +0000 Subject: [PATCH 120/496] Check type after optimize_rewrite_aggregate_function_with_if. --- .../RewriteAggregateFunctionWithIfPass.cpp | 32 ++++++++++++++++--- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++- 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index c1adf05ac76..a48e88132a6 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -42,7 +43,7 @@ public: if (lower_name.ends_with("if")) return; - auto & function_arguments_nodes = function_node->getArguments().getNodes(); + const auto & function_arguments_nodes = function_node->getArguments().getNodes(); if (function_arguments_nodes.size() != 1) return; @@ -50,6 +51,8 @@ public: if (!if_node || if_node->getFunctionName() != "if") return; + FunctionNodePtr replaced_node; + auto if_arguments_nodes = if_node->getArguments().getNodes(); auto * first_const_node = if_arguments_nodes[1]->as(); auto * second_const_node = if_arguments_nodes[2]->as(); @@ -75,8 +78,11 @@ public: new_arguments[0] = std::move(if_arguments_nodes[1]); new_arguments[1] = std::move(if_arguments_nodes[0]); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } else if (first_const_node) @@ -104,10 +110,26 @@ public: FunctionFactory::instance().get("not", getContext())->build(not_function->getArgumentColumns())); new_arguments[1] = std::move(not_function); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } + + if (!replaced_node) + return; + + auto prev_type = function_node->getResultType(); + auto curr_type = replaced_node->getResultType(); + if (!prev_type->equals(*curr_type)) + return; + + /// Just in case, CAST compatible aggregate function states. + if (WhichDataType(prev_type).isAggregateFunction() && !DataTypeAggregateFunction::strictEquals(prev_type, curr_type)) + node = createCastFunction(std::move(replaced_node), prev_type, getContext()); + else + node = std::move(replaced_node); } }; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1fe2554988..b1603bb18dd 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,11 +3239,13 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); + std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; + AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - + std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 4833b46a1a86bb1847d2520ea12ea4650c497abc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:43:30 +0000 Subject: [PATCH 121/496] Remove debug code --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1603bb18dd..b1fe2554988 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,13 +3239,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); - std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; - AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; + function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 1ba44252cd20ab660d374970257a1ceb438236dd Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 26 Jul 2024 18:33:50 +0000 Subject: [PATCH 122/496] turn sql to bash --- ...uery_views_log_background_thread.reference | 25 +----------- ...02572_query_views_log_background_thread.sh | 38 ++++++++++++++++++ ...2572_query_views_log_background_thread.sql | 40 ------------------- 3 files changed, 40 insertions(+), 63 deletions(-) create mode 100755 tests/queries/0_stateless/02572_query_views_log_background_thread.sh delete mode 100644 tests/queries/0_stateless/02572_query_views_log_background_thread.sql diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index f867fd0d085..d7f2272f5b4 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -1,25 +1,4 @@ --- { echoOn } -insert into buffer_02572 values (1); --- ensure that the flush was not direct -select * from buffer_02572; +OK +1 1 -select * from data_02572; -select * from copy_02572; --- we cannot use OPTIMIZE, this will attach query context, so let's wait -SET function_sleep_max_microseconds_per_block = 6000000; -select sleepEachRow(1) from numbers(3*2) format Null; -select sleepEachRow(1) from numbers(3*2) format Null; -system flush logs; -select count() > 0, lower(status::String), errorCodeToName(exception_code) - from system.query_views_log where - view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and - view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3 -; 1 queryfinish OK -select * from buffer_02572; -1 -select * from data_02572; -1 -select * from copy_02572; -1 diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh new file mode 100755 index 00000000000..a3e428e75c8 --- /dev/null +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +# INSERT buffer_02572 -> data_02572 -> copy_02572 +# ^^ +# push to system.query_views_log + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists buffer_02572; + drop table if exists data_02572; drop table if exists copy_02572; drop table if exists mv_02572;" + +${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" +${CLICKHOUSE_CLIENT} --query="create table data_02572 (key Int) engine=Memory();" +${CLICKHOUSE_CLIENT} --query="create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, 3, 3, 1, 1e9, 1, 1e9);" +${CLICKHOUSE_CLIENT} --query="create materialized view mv_02572 to copy_02572 as select * from data_02572;" + +${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);" + +# ensure that the flush was not direct +${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" + +# we cannot use OPTIMIZE, this will attach query context, so let's wait +for _ in {1..100}; do + $CLICKHOUSE_CLIENT -q "select * from data_02572;" | grep -q "1" && echo 'OK' && break + sleep 0.5 +done + + +${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" + +${CLICKHOUSE_CLIENT} --query="system flush logs;" +${CLICKHOUSE_CLIENT} --query="select count() > 0, lower(status::String), errorCodeToName(exception_code) + from system.query_views_log where + view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and + view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') + group by 2, 3;" \ No newline at end of file diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql deleted file mode 100644 index 2e9a62b71da..00000000000 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ /dev/null @@ -1,40 +0,0 @@ --- INSERT buffer_02572 -> data_02572 -> copy_02572 --- ^^ --- push to system.query_views_log - -drop table if exists buffer_02572; -drop table if exists data_02572; -drop table if exists copy_02572; -drop table if exists mv_02572; - -create table copy_02572 (key Int) engine=Memory(); -create table data_02572 (key Int) engine=Memory(); -create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, - /* never direct flush for flush from background thread */ - /* min_time= */ 3, 3, - 1, 1e9, - 1, 1e9); -create materialized view mv_02572 to copy_02572 as select * from data_02572; - --- { echoOn } -insert into buffer_02572 values (1); --- ensure that the flush was not direct -select * from buffer_02572; -select * from data_02572; -select * from copy_02572; --- we cannot use OPTIMIZE, this will attach query context, so let's wait -SET function_sleep_max_microseconds_per_block = 6000000; -select sleepEachRow(1) from numbers(3*2) format Null; -select sleepEachRow(1) from numbers(3*2) format Null; - -system flush logs; -select count() > 0, lower(status::String), errorCodeToName(exception_code) - from system.query_views_log where - view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and - view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3 -; - -select * from buffer_02572; -select * from data_02572; -select * from copy_02572; \ No newline at end of file From 870ec237bb427243388acbe5bca770241eeb7fbb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Jun 2024 14:14:11 +0200 Subject: [PATCH 123/496] Add ability to load dashboards for system.dashboards from config One of the obvious reasons is to allow rendering them with readonly user, which is not possible right now, due to usage of merge() function there. Another one, is to add some custom metrics. Note, that once set, they overrides the default dashboards preset. Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 2 + programs/server/config.xml | 25 ++++++++++ src/Interpreters/Context.cpp | 49 +++++++++++++++++++ src/Interpreters/Context.h | 4 ++ .../System/StorageSystemDashboards.cpp | 26 +++++++--- src/Storages/System/StorageSystemDashboards.h | 2 +- .../test_custom_dashboards/__init__.py | 0 .../configs/config.d/overrides.xml | 15 ++++++ .../test_custom_dashboards/test.py | 35 +++++++++++++ 9 files changed, 149 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_custom_dashboards/__init__.py create mode 100644 tests/integration/test_custom_dashboards/configs/config.d/overrides.xml create mode 100644 tests/integration/test_custom_dashboards/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..f8aea3ad10c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1582,6 +1582,8 @@ try global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); + global_context->setDashboardsConfig(config); + if (global_context->isServerCompletelyStarted()) { /// It does not make sense to reload anything before server has started. diff --git a/programs/server/config.xml b/programs/server/config.xml index 94825a55f67..5dedd78ff2a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1312,6 +1312,31 @@ event_date + INTERVAL 30 DAY + + + + - - create table views_max_insert_threads_null (a UInt64) Engine = Null - create materialized view views_max_insert_threads_mv Engine = Null AS select now() as ts, max(a) from views_max_insert_threads_null group by ts - - insert into views_max_insert_threads_null select * from numbers_mt(3000000000) settings max_threads = 16, max_insert_threads=16 - - drop table if exists views_max_insert_threads_null - drop table if exists views_max_insert_threads_mv - - + + + + + + + + + + From b2d8eaf1e6d67ab76f3e86cd4fd857e9535a9d20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 18:55:08 +0200 Subject: [PATCH 171/496] Debug TimerDescriptor --- src/Common/TimerDescriptor.cpp | 26 ++++++++++++++++++++++++-- src/IO/MMappedFileDescriptor.cpp | 2 -- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index b36ea4059cb..9a171ae9487 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -2,9 +2,11 @@ #include #include +#include #include #include +#include namespace DB @@ -89,9 +91,29 @@ void TimerDescriptor::drain() const /// A signal happened, need to retry. if (errno == EINTR) - continue; + { + /** This is to help with debugging. + * + * Sometimes reading from timer_fd blocks, which should not happen, because we opened it in a non-blocking mode. + * But it could be possible if a rogue 3rd-party library closed our file descriptor by mistake + * (for example by double closing due to the lack of exception safety or if it is a crappy code in plain C) + * and then another file descriptor is opened in its place. + * + * Let's try to get a name of this file descriptor and log it. + */ + LoggerPtr log = getLogger("TimerDescriptor"); - throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd"); + static constexpr ssize_t max_link_path_length = 256; + char link_path[max_link_path_length]; + ssize_t link_path_length = readlink(fmt::format("/proc/self/fd/{}", timer_fd).c_str(), link_path, max_link_path_length); + if (-1 == link_path_length) + throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot readlink for a timer_fd {}", timer_fd); + + LOG_TRACE(log, "Received EINTR while trying to drain a TimerDescriptor, fd {}: {}", timer_fd, std::string_view(link_path, link_path_length)); + continue; + } + + throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd {}", timer_fd); } chassert(res == sizeof(buf)); diff --git a/src/IO/MMappedFileDescriptor.cpp b/src/IO/MMappedFileDescriptor.cpp index a7eb8e4ede5..47f80005c9d 100644 --- a/src/IO/MMappedFileDescriptor.cpp +++ b/src/IO/MMappedFileDescriptor.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include From 45db7c85cf25f9b4b27cedd7464a786c53580d3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 29 Jul 2024 16:57:15 +0000 Subject: [PATCH 172/496] Remove has_single_port property from plan stream. --- src/Processors/QueryPlan/AggregatingStep.cpp | 2 -- src/Processors/QueryPlan/FillingStep.cpp | 8 +++----- src/Processors/QueryPlan/IQueryPlanStep.h | 6 +----- src/Processors/QueryPlan/ITransformingStep.cpp | 3 --- src/Processors/QueryPlan/ReadNothingStep.cpp | 2 +- 5 files changed, 5 insertions(+), 16 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f31de80b22d..8a5ed7fde65 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -134,7 +134,6 @@ AggregatingStep::AggregatingStep( { output_stream->sort_description = group_by_sort_description; output_stream->sort_scope = DataStream::SortScope::Global; - output_stream->has_single_port = true; } } @@ -147,7 +146,6 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_, { output_stream->sort_description = group_by_sort_description; output_stream->sort_scope = DataStream::SortScope::Global; - output_stream->has_single_port = true; } explicit_sorting_required_for_aggregation_in_order = false; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 65c9cf11661..81622389ada 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -39,12 +39,13 @@ FillingStep::FillingStep( , interpolate_description(interpolate_description_) , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { - if (!input_stream_.has_single_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); } void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + if (pipeline.getNumStreams() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) @@ -69,9 +70,6 @@ void FillingStep::describeActions(JSONBuilder::JSONMap & map) const void FillingStep::updateOutputStream() { - if (!input_streams.front().has_single_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); - output_stream = createOutputStream( input_streams.front(), FillingTransform::transformHeader(input_streams.front().header, sort_description), getDataStreamTraits()); } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index daca88fcceb..44eb7ea0c59 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -28,9 +28,6 @@ class DataStream public: Block header; - /// QueryPipeline has single port. Totals or extremes ports are not counted. - bool has_single_port = false; - /// Sorting scope. Please keep the mutual order (more strong mode should have greater value). enum class SortScope : uint8_t { @@ -51,8 +48,7 @@ public: bool hasEqualPropertiesWith(const DataStream & other) const { - return has_single_port == other.has_single_port - && sort_description == other.sort_description + return sort_description == other.sort_description && (sort_description.empty() || sort_scope == other.sort_scope); } diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index 9ecfdb0af22..3fa9d1b8308 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -20,9 +20,6 @@ DataStream ITransformingStep::createOutputStream( { DataStream output_stream{.header = std::move(output_header)}; - output_stream.has_single_port = stream_traits.returns_single_stream - || (input_stream.has_single_port && stream_traits.preserves_number_of_streams); - if (stream_traits.preserves_sorting) { output_stream.sort_description = input_stream.sort_description; diff --git a/src/Processors/QueryPlan/ReadNothingStep.cpp b/src/Processors/QueryPlan/ReadNothingStep.cpp index 253f3a5b980..3037172bbd4 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.cpp +++ b/src/Processors/QueryPlan/ReadNothingStep.cpp @@ -6,7 +6,7 @@ namespace DB { ReadNothingStep::ReadNothingStep(Block output_header) - : ISourceStep(DataStream{.header = std::move(output_header), .has_single_port = true}) + : ISourceStep(DataStream{.header = std::move(output_header)}) { } From 412268bf4e64b5c0df3980e5b8ccd2b078cf2177 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 17:05:46 +0000 Subject: [PATCH 173/496] Update reference --- ...dynamic_read_subcolumns_small.reference.j2 | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 index 3d814e1205a..be3f4e53990 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 @@ -25,26 +25,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -[[0]] \N \N [] str_10 \N str_10 [] -[[0,1]] \N \N [] +[[0]] \N \N [] str_11 \N str_11 [] -[[0,1,2]] \N \N [] +[[0,1]] \N \N [] str_12 \N str_12 [] -[[0,1,2,3]] \N \N [] +[[0,1,2]] \N \N [] str_13 \N str_13 [] -[[0,1,2,3,4]] \N \N [] +[[0,1,2,3]] \N \N [] str_14 \N str_14 [] -[[0,1,2,3,4,5]] \N \N [] +[[0,1,2,3,4]] \N \N [] str_15 \N str_15 [] -[[0,1,2,3,4,5,6]] \N \N [] +[[0,1,2,3,4,5]] \N \N [] str_16 \N str_16 [] -[[0,1,2,3,4,5,6,7]] \N \N [] +[[0,1,2,3,4,5,6]] \N \N [] str_17 \N str_17 [] -[[0,1,2,3,4,5,6,7,8]] \N \N [] +[[0,1,2,3,4,5,6,7]] \N \N [] str_18 \N str_18 [] -[[0,1,2,3,4,5,6,7,8,9]] \N \N [] +[[0,1,2,3,4,5,6,7,8]] \N \N [] str_19 \N str_19 [] +[[0,1,2,3,4,5,6,7,8,9]] \N \N [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] From 812a2b929938c293441f6a893adf96a00d469351 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 17:24:28 +0000 Subject: [PATCH 174/496] formatDateTime[InJodaSyntax]: make format string optional --- .../functions/type-conversion-functions.md | 12 ++++---- src/Functions/parseDateTime.cpp | 30 ++++++++++++------- .../02668_parse_datetime.reference | 6 +++- .../0_stateless/02668_parse_datetime.sql | 7 ++++- ...68_parse_datetime_in_joda_syntax.reference | 6 +++- .../02668_parse_datetime_in_joda_syntax.sql | 7 ++++- 6 files changed, 48 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 7cc2c022143..dc90697bd20 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -3467,13 +3467,13 @@ This function is the opposite operation of function [formatDateTime](../function **Syntax** ``` sql -parseDateTime(str, format[, timezone]) +parseDateTime(str[, format[, timezone]]) ``` **Arguments** -- `str` — the String to be parsed -- `format` — the format string +- `str` — The String to be parsed +- `format` — The format string. Optional. `%Y-%m-%d %H:%i:%s` if not specified. - `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** @@ -3516,13 +3516,13 @@ This function is the opposite operation of function [formatDateTimeInJodaSyntax] **Syntax** ``` sql -parseDateTimeInJodaSyntax(str, format[, timezone]) +parseDateTimeInJodaSyntax(str[, format[, timezone]]) ``` **Arguments** -- `str` — the String to be parsed -- `format` — the format string +- `str` — The String to be parsed +- `format` — The format string. Optional. `yyyy-MM-dd HH:mm:ss` if not specified. - `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index bdca0151bba..7ca10677be7 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -582,11 +582,11 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"time", static_cast(&isString), nullptr, "String"}, - {"format", static_cast(&isString), nullptr, "String"} + {"time", static_cast(&isString), nullptr, "String"} }; FunctionArgumentDescriptors optional_args{ + {"format", static_cast(&isString), nullptr, "String"}, {"timezone", static_cast(&isString), &isColumnConst, "const String"} }; @@ -2029,14 +2029,24 @@ namespace String getFormat(const ColumnsWithTypeAndName & arguments) const { - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!format_column) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second ('format') argument of function {}. Must be constant string.", - arguments[1].column->getName(), - getName()); - return format_column->getValue(); + if (arguments.size() == 1) + { + if constexpr (parse_syntax == ParseSyntax::MySQL) + return "%Y-%m-%d %H:%i:%s"; + else + return "yyyy-MM-dd HH:mm:ss"; + } + else + { + const auto * col_format = checkAndGetColumnConst(arguments[1].column.get()); + if (!col_format) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), + getName()); + return col_format->getValue(); + } } const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index d21a51ce70c..b67ca2d8b76 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -239,7 +239,7 @@ select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTi select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; 1 -- Error handling -select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Fuzzer crash bug #53715 select parseDateTime('', '', toString(number)) from numbers(13); -- { serverError ILLEGAL_COLUMN } @@ -270,3 +270,7 @@ select parseDateTime('8 13, 2022, 7:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); 2022-08-13 07:58:32 select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); 2022-08-13 07:58:32 +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); +1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 02ac0c5f35c..7b3aed60a4a 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -162,7 +162,7 @@ select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTi select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; -- Error handling -select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Fuzzer crash bug #53715 @@ -187,4 +187,9 @@ select parseDateTime('08 13, 2022, 07:58:32', '%m %e, %G, %k:%i:%s', 'UTC'); select parseDateTime('8 13, 2022, 7:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); + + -- { echoOff } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference index 9fbf105dc41..6f560577ab5 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -354,5 +354,9 @@ select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; 1 -- Error handling -select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); +1 diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index f5810d3d4c3..28d14607ba6 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -239,6 +239,11 @@ select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; -- Error handling -select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); + -- { echoOff } From 1c9d60ca972eab618b82704e14f1a680daed9c04 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 16:36:45 +0000 Subject: [PATCH 175/496] Refactoring --- programs/client/Client.h | 6 +- programs/local/LocalServer.h | 4 +- src/Client/ClientApplicationBase.cpp | 539 +++++++++++++++++++++++++++ src/Client/ClientApplicationBase.h | 54 +++ src/Client/ClientBase.cpp | 362 +----------------- src/Client/ClientBase.h | 73 +++- 6 files changed, 659 insertions(+), 379 deletions(-) create mode 100644 src/Client/ClientApplicationBase.cpp create mode 100644 src/Client/ClientApplicationBase.h diff --git a/programs/client/Client.h b/programs/client/Client.h index 9571440d6ba..7fdf77031ab 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -1,14 +1,16 @@ #pragma once -#include +#include namespace DB { -class Client : public ClientBase +class Client : public ClientApplicationBase { public: + using Arguments = ClientApplicationBase::Arguments; + Client() = default; void initialize(Poco::Util::Application & self) override; diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index ae9980311e1..b18a7a90961 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -21,7 +21,7 @@ namespace DB /// Lightweight Application for clickhouse-local /// No networking, no extra configs and working directories, no pid and status files, no dictionaries, no logging. /// Quiet mode by default -class LocalServer : public ClientBase, public Loggers +class LocalServer : public ClientApplicationBase, public Loggers { public: LocalServer() = default; diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp new file mode 100644 index 00000000000..59c98983694 --- /dev/null +++ b/src/Client/ClientApplicationBase.cpp @@ -0,0 +1,539 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include +#include "config.h" + +#include +#include +#include +#include +#include + +using namespace std::literals; + +namespace CurrentMetrics +{ + extern const Metric MemoryTracking; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_SET_SIGNAL_HANDLER; + extern const int UNRECOGNIZED_ARGUMENTS; +} + +static ClientInfo::QueryKind parseQueryKind(const String & query_kind) +{ + if (query_kind == "initial_query") + return ClientInfo::QueryKind::INITIAL_QUERY; + if (query_kind == "secondary_query") + return ClientInfo::QueryKind::SECONDARY_QUERY; + if (query_kind == "no_query") + return ClientInfo::QueryKind::NO_QUERY; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query kind {}", query_kind); +} + +/// This signal handler is set only for SIGINT and SIGQUIT. +void interruptSignalHandler(int signum) +{ + if (ClientApplicationBase::getInstance().tryStopQuery()) + safeExit(128 + signum); +} + +ClientApplicationBase::~ClientApplicationBase() = default; +ClientApplicationBase::ClientApplicationBase() : ClientBase(STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO, std::cin, std::cout, std::cerr) {} + +ClientApplicationBase & ClientApplicationBase::getInstance() +{ + return dynamic_cast(Poco::Util::Application::instance()); +} + +void ClientApplicationBase::setupSignalHandler() +{ + ClientApplicationBase::getInstance().stopQuery(); + + struct sigaction new_act; + memset(&new_act, 0, sizeof(new_act)); + + new_act.sa_handler = interruptSignalHandler; + new_act.sa_flags = 0; + +#if defined(OS_DARWIN) + sigemptyset(&new_act.sa_mask); +#else + if (sigemptyset(&new_act.sa_mask)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); +#endif + + if (sigaction(SIGINT, &new_act, nullptr)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); + + if (sigaction(SIGQUIT, &new_act, nullptr)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); +} + + +namespace +{ + +/// Define transparent hash to we can use +/// std::string_view with the containers +struct TransparentStringHash +{ + using is_transparent = void; + size_t operator()(std::string_view txt) const + { + return std::hash{}(txt); + } +}; + +/* + * This functor is used to parse command line arguments and replace dashes with underscores, + * allowing options to be specified using either dashes or underscores. + */ +class OptionsAliasParser +{ +public: + explicit OptionsAliasParser(const boost::program_options::options_description& options) + { + options_names.reserve(options.options().size()); + for (const auto& option : options.options()) + options_names.insert(option->long_name()); + } + + /* + * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options + * Implements boost::program_options::ext_parser logic + */ + std::pair operator()(const std::string & token) const + { + if (!token.starts_with("--")) + return {}; + std::string arg = token.substr(2); + + // divide token by '=' to separate key and value if options style=long_allow_adjacent + auto pos_eq = arg.find('='); + std::string key = arg.substr(0, pos_eq); + + if (options_names.contains(key)) + // option does not require any changes, because it is already correct + return {}; + + std::replace(key.begin(), key.end(), '-', '_'); + if (!options_names.contains(key)) + // after replacing '-' with '_' argument is still unknown + return {}; + + std::string value; + if (pos_eq != std::string::npos && pos_eq < arg.size()) + value = arg.substr(pos_eq + 1); + + return {key, value}; + } + +private: + std::unordered_set options_names; +}; + +} + +/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests +#if defined(__clang__) +#pragma clang optimize on +#endif +void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +{ + if (allow_repeated_settings) + addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); + else + addProgramOptions(cmd_settings, options_description.main_description.value()); + + if (allow_merge_tree_settings) + { + /// Add merge tree settings manually, because names of some settings + /// may clash. Query settings have higher priority and we just + /// skip ambiguous merge tree settings. + auto & main_options = options_description.main_description.value(); + + std::unordered_set> main_option_names; + for (const auto & option : main_options.options()) + main_option_names.insert(option->long_name()); + + for (const auto & setting : cmd_merge_tree_settings.all()) + { + const auto add_setting = [&](const std::string_view name) + { + if (auto it = main_option_names.find(name); it != main_option_names.end()) + return; + + if (allow_repeated_settings) + addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); + else + addProgramOption(cmd_merge_tree_settings, main_options, name, setting); + }; + + const auto & setting_name = setting.getName(); + + add_setting(setting_name); + + const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); + if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) + { + for (const auto alias : it->second) + { + add_setting(alias); + } + } + } + } + + /// Parse main commandline options. + auto parser = po::command_line_parser(arguments) + .options(options_description.main_description.value()) + .extra_parser(OptionsAliasParser(options_description.main_description.value())) + .allow_unregistered(); + po::parsed_options parsed = parser.run(); + + /// Check unrecognized options without positional options. + auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); + if (!unrecognized_options.empty()) + { + auto hints = this->getHints(unrecognized_options[0]); + if (!hints.empty()) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", + unrecognized_options[0], toString(hints)); + + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); + } + + /// Check positional options. + for (const auto & op : parsed.options) + { + if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") + && !op.original_tokens[0].empty() && !op.value.empty()) + { + /// Two special cases for better usability: + /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" + /// These are relevant for interactive usage - user-friendly, but questionable in general. + /// In case of ambiguity or for scripts, prefer using proper options. + + const auto & token = op.original_tokens[0]; + po::variable_value value(boost::any(op.value), false); + + const char * option; + if (token.contains(' ')) + option = "query"; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + + if (!options.emplace(option, value).second) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + } + } + + po::store(parsed, options); +} + +void ClientApplicationBase::addMultiquery(std::string_view query, Arguments & common_arguments) const +{ + common_arguments.emplace_back("--multiquery"); + common_arguments.emplace_back("-q"); + common_arguments.emplace_back(query); +} + +Poco::Util::LayeredConfiguration & ClientApplicationBase::getClientConfiguration() +{ + return config(); +} + +void ClientApplicationBase::init(int argc, char ** argv) +{ + namespace po = boost::program_options; + + /// Don't parse options with Poco library, we prefer neat boost::program_options. + stopOptionsProcessing(); + + stdin_is_a_tty = isatty(STDIN_FILENO); + stdout_is_a_tty = isatty(STDOUT_FILENO); + stderr_is_a_tty = isatty(STDERR_FILENO); + terminal_width = getTerminalWidth(); + + std::vector external_tables_arguments; + Arguments common_arguments = {""}; /// 0th argument is ignored. + std::vector hosts_and_ports_arguments; + + if (argc) + argv0 = argv[0]; + readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments); + + /// Support for Unicode dashes + /// Interpret Unicode dashes as default double-hyphen + for (auto & arg : common_arguments) + { + // replace em-dash(U+2014) + boost::replace_all(arg, "—", "--"); + // replace en-dash(U+2013) + boost::replace_all(arg, "–", "--"); + // replace mathematical minus(U+2212) + boost::replace_all(arg, "−", "--"); + } + + + OptionsDescription options_description; + options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width)); + + /// Common options for clickhouse-client and clickhouse-local. + options_description.main_description->add_options() + ("help", "print usage summary, combine with --verbose to display all options") + ("verbose", "print query and other debugging info") + ("version,V", "print version information and exit") + ("version-clean", "print version in machine-readable format and exit") + + ("config-file,C", po::value(), "config-file path") + + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") + ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") + ("multiquery,n", "Obsolete, does nothing") + ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") + ("database,d", po::value(), "database") + ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") + ("query_id", po::value(), "query_id") + + ("history_file", po::value(), "path to history file") + + ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") + + ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") + ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") + ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") + + ("echo", "in batch mode, print query before execution") + + ("log-level", po::value(), "log level") + ("server_logs_file", po::value(), "put server logs into specified file") + + ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") + + ("format,f", po::value(), "default output format (and input format for clickhouse-local)") + ("output-format", po::value(), "default output format (this option has preference over --format)") + + ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") + ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") + + ("ignore-error", "do not stop processing when an error occurs") + ("stacktrace", "print stack traces of exceptions") + ("hardware-utilization", "print hardware utilization information in progress bar") + ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") + ("profile-events-delay-ms", po::value()->default_value(profile_events.delay_ms), "Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet)") + ("processed-rows", "print the number of locally processed rows") + + ("interactive", "Process queries-file or --query query and start interactive mode") + ("pager", po::value(), "Pipe all output into this command (less or similar)") + ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") + + ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") + + ("client_logs_file", po::value(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)") + ; + + addOptions(options_description); + + OptionsDescription options_description_non_verbose = options_description; + + auto getter = [](const auto & op) + { + String op_long_name = op->long_name(); + return "--" + String(op_long_name); + }; + + if (options_description.main_description) + { + const auto & main_options = options_description.main_description->options(); + std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); + } + + if (options_description.external_description) + { + const auto & external_options = options_description.external_description->options(); + std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); + } + + po::variables_map options; + parseAndCheckOptions(options_description, options, common_arguments); + po::notify(options); + + if (options.count("version") || options.count("V")) + { + showClientVersion(); + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + if (options.count("version-clean")) + { + output_stream << VERSION_STRING; + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + if (options.count("verbose")) + getClientConfiguration().setBool("verbose", true); + + /// Output of help message. + if (options.count("help") + || (options.count("host") && options["host"].as() == "elp")) /// If user writes -help instead of --help. + { + if (getClientConfiguration().getBool("verbose", false)) + printHelpMessage(options_description, true); + else + printHelpMessage(options_description_non_verbose, false); + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + /// Common options for clickhouse-client and clickhouse-local. + + /// Output execution time to stderr in batch mode. + if (options.count("time")) + getClientConfiguration().setBool("print-time-to-stderr", true); + if (options.count("memory-usage")) + { + const auto & memory_usage_mode = options["memory-usage"].as(); + if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); + getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); + } + + if (options.count("query")) + queries = options["query"].as>(); + if (options.count("query_id")) + getClientConfiguration().setString("query_id", options["query_id"].as()); + if (options.count("database")) + getClientConfiguration().setString("database", options["database"].as()); + if (options.count("config-file")) + getClientConfiguration().setString("config-file", options["config-file"].as()); + if (options.count("queries-file")) + queries_files = options["queries-file"].as>(); + if (options.count("multiline")) + getClientConfiguration().setBool("multiline", true); + if (options.count("ignore-error")) + getClientConfiguration().setBool("ignore-error", true); + if (options.count("format")) + getClientConfiguration().setString("format", options["format"].as()); + if (options.count("output-format")) + getClientConfiguration().setString("output-format", options["output-format"].as()); + if (options.count("vertical")) + getClientConfiguration().setBool("vertical", true); + if (options.count("stacktrace")) + getClientConfiguration().setBool("stacktrace", true); + if (options.count("print-profile-events")) + getClientConfiguration().setBool("print-profile-events", true); + if (options.count("profile-events-delay-ms")) + getClientConfiguration().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as()); + /// Whether to print the number of processed rows at + if (options.count("processed-rows")) + getClientConfiguration().setBool("print-num-processed-rows", true); + if (options.count("progress")) + { + switch (options["progress"].as()) + { + case DEFAULT: + getClientConfiguration().setString("progress", "default"); + break; + case OFF: + getClientConfiguration().setString("progress", "off"); + break; + case TTY: + getClientConfiguration().setString("progress", "tty"); + break; + case ERR: + getClientConfiguration().setString("progress", "err"); + break; + } + } + if (options.count("echo")) + getClientConfiguration().setBool("echo", true); + if (options.count("disable_suggestion")) + getClientConfiguration().setBool("disable_suggestion", true); + if (options.count("wait_for_suggestions_to_load")) + getClientConfiguration().setBool("wait_for_suggestions_to_load", true); + if (options.count("suggestion_limit")) + getClientConfiguration().setInt("suggestion_limit", options["suggestion_limit"].as()); + if (options.count("highlight")) + getClientConfiguration().setBool("highlight", options["highlight"].as()); + if (options.count("history_file")) + getClientConfiguration().setString("history_file", options["history_file"].as()); + if (options.count("interactive")) + getClientConfiguration().setBool("interactive", true); + if (options.count("pager")) + getClientConfiguration().setString("pager", options["pager"].as()); + + if (options.count("log-level")) + Poco::Logger::root().setLevel(options["log-level"].as()); + if (options.count("server_logs_file")) + server_logs_file = options["server_logs_file"].as(); + + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); + query_kind = parseQueryKind(options["query_kind"].as()); + profile_events.print = options.count("print-profile-events"); + profile_events.delay_ms = options["profile-events-delay-ms"].as(); + + processOptions(options_description, options, external_tables_arguments, hosts_and_ports_arguments); + { + std::unordered_set alias_names; + alias_names.reserve(options_description.main_description->options().size()); + for (const auto& option : options_description.main_description->options()) + alias_names.insert(option->long_name()); + argsToConfig(common_arguments, getClientConfiguration(), 100, &alias_names); + } + + clearPasswordFromCommandLine(argc, argv); + + /// Limit on total memory usage + std::string max_client_memory_usage = getClientConfiguration().getString("max_memory_usage_in_client", "0" /*default value*/); + if (max_client_memory_usage != "0") + { + UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); + + total_memory_tracker.setHardLimit(max_client_memory_usage_int); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + } + + /// Print stacktrace in case of crash + HandledSignals::instance().setupTerminateHandler(); + HandledSignals::instance().setupCommonDeadlySignalHandlers(); + /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't + /// have an option for client to shutdown gracefully. + + fatal_channel_ptr = new Poco::SplitterChannel; + fatal_console_channel_ptr = new Poco::ConsoleChannel; + fatal_channel_ptr->addChannel(fatal_console_channel_ptr); + if (options.count("client_logs_file")) + { + fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as()); + fatal_channel_ptr->addChannel(fatal_file_channel_ptr); + } + + fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); + signal_listener = std::make_unique(nullptr, fatal_log); + signal_listener_thread.start(*signal_listener); + +#if USE_GWP_ASAN + GWPAsan::initFinished(); +#endif + +} + + +} diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h new file mode 100644 index 00000000000..217fa29c3f4 --- /dev/null +++ b/src/Client/ClientApplicationBase.h @@ -0,0 +1,54 @@ +#pragma once + + +#include +#include +#include +#include +#include +#include + +#include + +namespace po = boost::program_options; + +namespace DB +{ + +void interruptSignalHandler(int signum); + +/** + * The base class for client appliucations such as + * clickhouse-client or clickhouse-local. + * The main purpose and responsibility of it is dealing with + * application-specific stuff such as command line arguments parsing + * and setting up signal handlers, so queries will be cancelled after + * Ctrl+C is pressed. + */ +class ClientApplicationBase : public ClientBase, public Poco::Util::Application, public IHints<2> +{ +public: + using ClientBase::processOptions; + using Arguments = ClientBase::Arguments; + + static ClientApplicationBase & getInstance(); + + ClientApplicationBase(); + ~ClientApplicationBase() override; + + void init(int argc, char ** argv); + std::vector getAllRegisteredNames() const override { return cmd_options; } + +protected: + Poco::Util::LayeredConfiguration & getClientConfiguration() override; + void setupSignalHandler() override; + void addMultiquery(std::string_view query, Arguments & common_arguments) const; + +private: + void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); + + std::vector cmd_options; +}; + + +} diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a88eed25db1..9cf3b955d26 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -5,7 +5,6 @@ #include #include -#include #include #include #include @@ -17,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -158,17 +156,6 @@ std::istream& operator>> (std::istream & in, ProgressOption & progress) return in; } -static ClientInfo::QueryKind parseQueryKind(const String & query_kind) -{ - if (query_kind == "initial_query") - return ClientInfo::QueryKind::INITIAL_QUERY; - if (query_kind == "secondary_query") - return ClientInfo::QueryKind::SECONDARY_QUERY; - if (query_kind == "no_query") - return ClientInfo::QueryKind::NO_QUERY; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query kind {}", query_kind); -} - static void incrementProfileEventsBlock(Block & dst, const Block & src) { if (!dst) @@ -269,36 +256,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) dst.setColumns(std::move(mutable_columns)); } - -std::atomic exit_after_signals = 0; - -class QueryInterruptHandler : private boost::noncopyable -{ -public: - /// Store how much interrupt signals can be before stopping the query - /// by default stop after the first interrupt signal. - static void start(Int32 signals_before_stop = 1) { exit_after_signals.store(signals_before_stop); } - - /// Set value not greater then 0 to mark the query as stopped. - static void stop() { exit_after_signals.store(0); } - - /// Return true if the query was stopped. - /// Query was stopped if it received at least "signals_before_stop" interrupt signals. - static bool try_stop() { return exit_after_signals.fetch_sub(1) <= 0; } - static bool cancelled() { return exit_after_signals.load() <= 0; } - - /// Return how much interrupt signals remain before stop. - static Int32 cancelled_status() { return exit_after_signals.load(); } -}; - -/// This signal handler is set for SIGINT and SIGQUIT. -void interruptSignalHandler(int signum) -{ - if (QueryInterruptHandler::try_stop()) - safeExit(128 + signum); -} - - /// To cancel the query on local format error. class LocalFormatError : public DB::Exception { @@ -345,31 +302,6 @@ ClientBase::ClientBase( terminal_width = getTerminalWidth(in_fd, err_fd); } -void ClientBase::setupSignalHandler() -{ - QueryInterruptHandler::stop(); - - struct sigaction new_act; - memset(&new_act, 0, sizeof(new_act)); - - new_act.sa_handler = interruptSignalHandler; - new_act.sa_flags = 0; - -#if defined(OS_DARWIN) - sigemptyset(&new_act.sa_mask); -#else - if (sigemptyset(&new_act.sa_mask)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); -#endif - - if (sigaction(SIGINT, &new_act, nullptr)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); - - if (sigaction(SIGQUIT, &new_act, nullptr)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); -} - - ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements) { std::unique_ptr parser; @@ -1113,8 +1045,8 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa { try { - QueryInterruptHandler::start(signals_before_stop); - SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + query_interrupt_handler.start(signals_before_stop); + SCOPE_EXIT({ query_interrupt_handler.stop(); }); connection->sendQuery( connection_parameters.timeouts, @@ -1178,13 +1110,13 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b /// to avoid losing sync. if (!cancelled) { - if (partial_result_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) + if (partial_result_on_first_cancel && query_interrupt_handler.cancelled_status() == signals_before_stop - 1) { connection->sendCancel(); /// First cancel reading request was sent. Next requests will only be with a full cancel partial_result_on_first_cancel = false; } - else if (QueryInterruptHandler::cancelled()) + else if (query_interrupt_handler.cancelled()) { cancelQuery(); } @@ -1563,8 +1495,8 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars return; } - QueryInterruptHandler::start(); - SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + query_interrupt_handler.start(); + SCOPE_EXIT({ query_interrupt_handler.stop(); }); connection->sendQuery( connection_parameters.timeouts, @@ -1775,7 +1707,7 @@ try Block block; while (executor.pull(block)) { - if (!cancelled && QueryInterruptHandler::cancelled()) + if (!cancelled && query_interrupt_handler.cancelled()) { cancelQuery(); executor.cancel(); @@ -2857,7 +2789,6 @@ void ClientBase::runLibFuzzer() void ClientBase::runLibFuzzer() {} #endif - void ClientBase::clearTerminal() { /// Clear from cursor until end of screen. @@ -2867,288 +2798,9 @@ void ClientBase::clearTerminal() output_stream << "\033[0J" "\033[?25h"; } - void ClientBase::showClientVersion() { output_stream << VERSION_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; } -void ClientBase::init(int argc, char ** argv) -{ - namespace po = boost::program_options; - - /// Don't parse options with Poco library, we prefer neat boost::program_options. - stopOptionsProcessing(); - - stdin_is_a_tty = isatty(STDIN_FILENO); - stdout_is_a_tty = isatty(STDOUT_FILENO); - stderr_is_a_tty = isatty(STDERR_FILENO); - terminal_width = getTerminalWidth(); - - std::vector external_tables_arguments; - Arguments common_arguments = {""}; /// 0th argument is ignored. - std::vector hosts_and_ports_arguments; - - if (argc) - argv0 = argv[0]; - readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments); - - /// Support for Unicode dashes - /// Interpret Unicode dashes as default double-hyphen - for (auto & arg : common_arguments) - { - // replace em-dash(U+2014) - boost::replace_all(arg, "—", "--"); - // replace en-dash(U+2013) - boost::replace_all(arg, "–", "--"); - // replace mathematical minus(U+2212) - boost::replace_all(arg, "−", "--"); - } - - - OptionsDescription options_description; - options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width)); - - /// Common options for clickhouse-client and clickhouse-local. - options_description.main_description->add_options() - ("help", "print usage summary, combine with --verbose to display all options") - ("verbose", "print query and other debugging info") - ("version,V", "print version information and exit") - ("version-clean", "print version in machine-readable format and exit") - - ("config-file,C", po::value(), "config-file path") - - ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") - ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") - ("multiquery,n", "Obsolete, does nothing") - ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") - ("database,d", po::value(), "database") - ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") - ("query_id", po::value(), "query_id") - - ("history_file", po::value(), "path to history file") - - ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") - - ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") - ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") - ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") - ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") - - ("echo", "in batch mode, print query before execution") - - ("log-level", po::value(), "log level") - ("server_logs_file", po::value(), "put server logs into specified file") - - ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") - - ("format,f", po::value(), "default output format (and input format for clickhouse-local)") - ("output-format", po::value(), "default output format (this option has preference over --format)") - - ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") - ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") - - ("ignore-error", "do not stop processing when an error occurs") - ("stacktrace", "print stack traces of exceptions") - ("hardware-utilization", "print hardware utilization information in progress bar") - ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") - ("profile-events-delay-ms", po::value()->default_value(profile_events.delay_ms), "Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet)") - ("processed-rows", "print the number of locally processed rows") - - ("interactive", "Process queries-file or --query query and start interactive mode") - ("pager", po::value(), "Pipe all output into this command (less or similar)") - ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") - - ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") - - ("client_logs_file", po::value(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)") - ; - - addOptions(options_description); - - OptionsDescription options_description_non_verbose = options_description; - - auto getter = [](const auto & op) - { - String op_long_name = op->long_name(); - return "--" + String(op_long_name); - }; - - if (options_description.main_description) - { - const auto & main_options = options_description.main_description->options(); - std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); - } - - if (options_description.external_description) - { - const auto & external_options = options_description.external_description->options(); - std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); - } - - po::variables_map options; - parseAndCheckOptions(options_description, options, common_arguments); - po::notify(options); - - if (options.count("version") || options.count("V")) - { - showClientVersion(); - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - if (options.count("version-clean")) - { - output_stream << VERSION_STRING; - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - if (options.count("verbose")) - getClientConfiguration().setBool("verbose", true); - - /// Output of help message. - if (options.count("help") - || (options.count("host") && options["host"].as() == "elp")) /// If user writes -help instead of --help. - { - if (getClientConfiguration().getBool("verbose", false)) - printHelpMessage(options_description, true); - else - printHelpMessage(options_description_non_verbose, false); - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - /// Common options for clickhouse-client and clickhouse-local. - - /// Output execution time to stderr in batch mode. - if (options.count("time")) - getClientConfiguration().setBool("print-time-to-stderr", true); - if (options.count("memory-usage")) - { - const auto & memory_usage_mode = options["memory-usage"].as(); - if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); - getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); - } - - if (options.count("query")) - queries = options["query"].as>(); - if (options.count("query_id")) - getClientConfiguration().setString("query_id", options["query_id"].as()); - if (options.count("database")) - getClientConfiguration().setString("database", options["database"].as()); - if (options.count("config-file")) - getClientConfiguration().setString("config-file", options["config-file"].as()); - if (options.count("queries-file")) - queries_files = options["queries-file"].as>(); - if (options.count("multiline")) - getClientConfiguration().setBool("multiline", true); - if (options.count("ignore-error")) - getClientConfiguration().setBool("ignore-error", true); - if (options.count("format")) - getClientConfiguration().setString("format", options["format"].as()); - if (options.count("output-format")) - getClientConfiguration().setString("output-format", options["output-format"].as()); - if (options.count("vertical")) - getClientConfiguration().setBool("vertical", true); - if (options.count("stacktrace")) - getClientConfiguration().setBool("stacktrace", true); - if (options.count("print-profile-events")) - getClientConfiguration().setBool("print-profile-events", true); - if (options.count("profile-events-delay-ms")) - getClientConfiguration().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as()); - /// Whether to print the number of processed rows at - if (options.count("processed-rows")) - getClientConfiguration().setBool("print-num-processed-rows", true); - if (options.count("progress")) - { - switch (options["progress"].as()) - { - case DEFAULT: - getClientConfiguration().setString("progress", "default"); - break; - case OFF: - getClientConfiguration().setString("progress", "off"); - break; - case TTY: - getClientConfiguration().setString("progress", "tty"); - break; - case ERR: - getClientConfiguration().setString("progress", "err"); - break; - } - } - if (options.count("echo")) - getClientConfiguration().setBool("echo", true); - if (options.count("disable_suggestion")) - getClientConfiguration().setBool("disable_suggestion", true); - if (options.count("wait_for_suggestions_to_load")) - getClientConfiguration().setBool("wait_for_suggestions_to_load", true); - if (options.count("suggestion_limit")) - getClientConfiguration().setInt("suggestion_limit", options["suggestion_limit"].as()); - if (options.count("highlight")) - getClientConfiguration().setBool("highlight", options["highlight"].as()); - if (options.count("history_file")) - getClientConfiguration().setString("history_file", options["history_file"].as()); - if (options.count("interactive")) - getClientConfiguration().setBool("interactive", true); - if (options.count("pager")) - getClientConfiguration().setString("pager", options["pager"].as()); - - if (options.count("log-level")) - Poco::Logger::root().setLevel(options["log-level"].as()); - if (options.count("server_logs_file")) - server_logs_file = options["server_logs_file"].as(); - - query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); - query_kind = parseQueryKind(options["query_kind"].as()); - profile_events.print = options.count("print-profile-events"); - profile_events.delay_ms = options["profile-events-delay-ms"].as(); - - processOptions(options_description, options, external_tables_arguments, hosts_and_ports_arguments); - { - std::unordered_set alias_names; - alias_names.reserve(options_description.main_description->options().size()); - for (const auto& option : options_description.main_description->options()) - alias_names.insert(option->long_name()); - argsToConfig(common_arguments, getClientConfiguration(), 100, &alias_names); - } - - clearPasswordFromCommandLine(argc, argv); - - /// Limit on total memory usage - std::string max_client_memory_usage = getClientConfiguration().getString("max_memory_usage_in_client", "0" /*default value*/); - if (max_client_memory_usage != "0") - { - UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); - - total_memory_tracker.setHardLimit(max_client_memory_usage_int); - total_memory_tracker.setDescription("(total)"); - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - } - - /// Print stacktrace in case of crash - HandledSignals::instance().setupTerminateHandler(); - HandledSignals::instance().setupCommonDeadlySignalHandlers(); - /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't - /// have an option for client to shutdown gracefully. - - fatal_channel_ptr = new Poco::SplitterChannel; - fatal_console_channel_ptr = new Poco::ConsoleChannel; - fatal_channel_ptr->addChannel(fatal_console_channel_ptr); - if (options.count("client_logs_file")) - { - fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as()); - fatal_channel_ptr->addChannel(fatal_file_channel_ptr); - } - - fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); - signal_listener = std::make_unique(nullptr, fatal_log); - signal_listener_thread.start(*signal_listener); - -#if USE_GWP_ASAN - GWPAsan::initFinished(); -#endif - -} - } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 7689744a373..557ac30d27c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -1,26 +1,30 @@ #pragma once -#include -#include "Common/NamePrompter.h" -#include -#include + +#include +#include #include +#include #include #include -#include #include #include -#include #include #include #include #include -#include -#include -#include -#include -#include +#include +#include #include +#include +#include + +#include + +#include +#include +#include +#include namespace po = boost::program_options; @@ -64,9 +68,16 @@ std::istream& operator>> (std::istream & in, ProgressOption & progress); class InternalTextLogs; class WriteBufferFromFileDescriptor; -class ClientBase : public Poco::Util::Application, public IHints<2> +/** + * The base class which encapsulates the core functionality of a client. + * Can be used in a standalone application (clickhouse-client or clickhouse-local), + * or be embedded into server. + * Always keep in mind that there can be several instances of this class within + * a process. Thus, it cannot keep its state in global shared variables or even use them. + * The best example - std::cin, std::cout and std::cerr. + */ +class ClientBase { - public: using Arguments = std::vector; @@ -79,12 +90,11 @@ public: std::ostream & output_stream_ = std::cout, std::ostream & error_stream_ = std::cerr ); + virtual ~ClientBase(); - ~ClientBase() override; + bool tryStopQuery() { return query_interrupt_handler.tryStop(); } + void stopQuery() { return query_interrupt_handler.stop(); } - void init(int argc, char ** argv); - - std::vector getAllRegisteredNames() const override { return cmd_options; } ASTPtr parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements); protected: @@ -114,7 +124,7 @@ protected: ASTPtr parsed_query, std::optional echo_query_ = {}, bool report_error = false); static void adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth, uint32_t max_parser_backtracks); - static void setupSignalHandler(); + virtual void setupSignalHandler() = 0; bool executeMultiQuery(const String & all_queries_text); MultiQueryProcessingStage analyzeMultiQueryText( @@ -188,7 +198,6 @@ private: String prompt() const; void resetOutput(); - void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); void updateSuggest(const ASTPtr & ast); @@ -196,6 +205,31 @@ private: bool addMergeTreeSettings(ASTCreateQuery & ast_create); protected: + + class QueryInterruptHandler : private boost::noncopyable + { + public: + /// Store how much interrupt signals can be before stopping the query + /// by default stop after the first interrupt signal. + void start(Int32 signals_before_stop = 1) { exit_after_signals.store(signals_before_stop); } + + /// Set value not greater then 0 to mark the query as stopped. + void stop() { exit_after_signals.store(0); } + + /// Return true if the query was stopped. + /// Query was stopped if it received at least "signals_before_stop" interrupt signals. + bool tryStop() { return exit_after_signals.fetch_sub(1) <= 0; } + bool cancelled() { return exit_after_signals.load() <= 0; } + + /// Return how much interrupt signals remain before stop. + Int32 cancelled_status() { return exit_after_signals.load(); } + + private: + std::atomic exit_after_signals = 0; + }; + + QueryInterruptHandler query_interrupt_handler; + static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); bool processMultiQueryFromFile(const String & file_name); @@ -239,7 +273,6 @@ protected: std::vector queries; /// Queries passed via '--query' std::vector queries_files; /// If not empty, queries will be read from these files std::vector interleave_queries_files; /// If not empty, run queries from these files before processing every file from 'queries_files'. - std::vector cmd_options; bool stdin_is_a_tty = false; /// stdin is a terminal. bool stdout_is_a_tty = false; /// stdout is a terminal. From 9186e647eb672283b8cb95d2fc152e0994f6df6f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 17:05:09 +0000 Subject: [PATCH 176/496] Fix style --- src/Client/ClientBase.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9cf3b955d26..56685f9d3f4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -88,12 +88,6 @@ namespace fs = std::filesystem; using namespace std::literals; - -namespace CurrentMetrics -{ - extern const Metric MemoryTracking; -} - namespace DB { From af2c9fcaaf4e2f38c9db105c246ee24b095b256f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 18:18:42 +0000 Subject: [PATCH 177/496] Skip file --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 380656cd1ca..3c959617d02 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -318,6 +318,7 @@ std_cerr_cout_excludes=( src/Interpreters/Context.cpp # IProcessor::dump() src/Processors/IProcessor.cpp + src/Client/ClientApplicationBase.cpp src/Client/ClientBase.cpp src/Client/LineReader.cpp src/Client/QueryFuzzer.cpp From 8ba85074e74f403b3d5106f6ef811019075cefb4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 19 Jul 2024 12:25:17 +0000 Subject: [PATCH 178/496] Fix build --- src/Client/ClientApplicationBase.cpp | 16 +++++++++++++++- src/Client/ClientApplicationBase.h | 11 +++++++++++ src/Client/ClientBase.cpp | 14 +------------- src/Client/ClientBase.h | 10 ---------- 4 files changed, 27 insertions(+), 24 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 59c98983694..6b19898ef5c 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include "config.h" @@ -51,7 +52,20 @@ void interruptSignalHandler(int signum) safeExit(128 + signum); } -ClientApplicationBase::~ClientApplicationBase() = default; +ClientApplicationBase::~ClientApplicationBase() +{ + try + { + writeSignalIDtoSignalPipe(SignalListener::StopThread); + signal_listener_thread.join(); + HandledSignals::instance().reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + ClientApplicationBase::ClientApplicationBase() : ClientBase(STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO, std::cin, std::cout, std::cerr) {} ClientApplicationBase & ClientApplicationBase::getInstance() diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h index 217fa29c3f4..771bb948cb7 100644 --- a/src/Client/ClientApplicationBase.h +++ b/src/Client/ClientApplicationBase.h @@ -6,6 +6,10 @@ #include #include #include +#include +#include +#include + #include #include @@ -48,6 +52,13 @@ private: void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); std::vector cmd_options; + + LoggerPtr fatal_log; + Poco::AutoPtr fatal_channel_ptr; + Poco::AutoPtr fatal_console_channel_ptr; + Poco::AutoPtr fatal_file_channel_ptr; + Poco::Thread signal_listener_thread; + std::unique_ptr signal_listener; }; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 56685f9d3f4..85dfb767e75 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -258,19 +258,7 @@ public: }; -ClientBase::~ClientBase() -{ - try - { - writeSignalIDtoSignalPipe(SignalListener::StopThread); - signal_listener_thread.join(); - HandledSignals::instance().reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} +ClientBase::~ClientBase() = default; ClientBase::ClientBase( int in_fd_, diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 557ac30d27c..304d8c4b890 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -9,9 +9,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -253,13 +250,6 @@ protected: /// Client context is a context used only by the client to parse queries, process query parameters and to connect to clickhouse-server. ContextMutablePtr client_context; - LoggerPtr fatal_log; - Poco::AutoPtr fatal_channel_ptr; - Poco::AutoPtr fatal_console_channel_ptr; - Poco::AutoPtr fatal_file_channel_ptr; - Poco::Thread signal_listener_thread; - std::unique_ptr signal_listener; - bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool delayed_interactive = false; From 49dc30d5c28392d361ce0ef1e18f7db73841617f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 24 Jul 2024 21:42:36 +0000 Subject: [PATCH 179/496] Small adjustement --- src/Client/ClientApplicationBase.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 6b19898ef5c..0306468d084 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -48,8 +48,13 @@ static ClientInfo::QueryKind parseQueryKind(const String & query_kind) /// This signal handler is set only for SIGINT and SIGQUIT. void interruptSignalHandler(int signum) { - if (ClientApplicationBase::getInstance().tryStopQuery()) - safeExit(128 + signum); + /// Signal handler might be called even before the setup is fully finished + /// and client application started to process the query. + /// Because of that we have to manually check it. + if (auto * instance = ClientApplicationBase::instanceRawPtr(); instance) + if (auto * base = dynamic_cast(instance); base) + if (base->tryStopQuery()) + safeExit(128 + signum); } ClientApplicationBase::~ClientApplicationBase() From c7c1f10720cd194d85de6d81156cbd37304ab52b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 24 Jul 2024 21:45:16 +0000 Subject: [PATCH 180/496] Added new method --- base/poco/Util/include/Poco/Util/Application.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index c8d18e1bce9..d1a2021eb67 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -261,6 +261,11 @@ namespace Util /// /// Throws a NullPointerException if no Application instance exists. + static Application * instanceRawPtr(); + /// Returns a raw pointer to the Application sigleton. + /// + /// The caller should check whether the result is nullptr. + const Poco::Timestamp & startTime() const; /// Returns the application start time (UTC). @@ -448,6 +453,12 @@ namespace Util } + inline Application * Application::instanceRawPtr() + { + return _pInstance; + } + + inline const Poco::Timestamp & Application::startTime() const { return _startTime; From 6f068639db627944aaab978c79866ad5a2a234e7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Jul 2024 16:38:26 +0000 Subject: [PATCH 181/496] Better --- src/Client/ClientApplicationBase.h | 1 - src/Client/ClientBase.h | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h index 771bb948cb7..3663271dd25 100644 --- a/src/Client/ClientApplicationBase.h +++ b/src/Client/ClientApplicationBase.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 304d8c4b890..175ebe97075 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -2,6 +2,7 @@ #include +#include #include #include #include @@ -9,9 +10,13 @@ #include #include #include +#include +#include +#include #include #include #include + #include #include #include From a457db34b216ea987a4875a3cbeb5363878d5d5a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Jul 2024 16:58:41 +0000 Subject: [PATCH 182/496] Fixed a typo --- base/poco/Util/include/Poco/Util/Application.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index d1a2021eb67..786e331fe73 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -262,7 +262,7 @@ namespace Util /// Throws a NullPointerException if no Application instance exists. static Application * instanceRawPtr(); - /// Returns a raw pointer to the Application sigleton. + /// Returns a raw pointer to the Application singleton. /// /// The caller should check whether the result is nullptr. From 2f255dc68d2cffb8bc17efc153a9e75a9166675d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 01:11:40 +0200 Subject: [PATCH 183/496] Fix clang-tidy --- src/Client/ClientBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 175ebe97075..1a23b6b1363 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -95,7 +95,7 @@ public: virtual ~ClientBase(); bool tryStopQuery() { return query_interrupt_handler.tryStop(); } - void stopQuery() { return query_interrupt_handler.stop(); } + void stopQuery() { query_interrupt_handler.stop(); } ASTPtr parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements); From 9b4accebb3d19789e69b422ae2f235149e453f94 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 15:47:52 +0000 Subject: [PATCH 184/496] Fix build --- src/Client/ClientApplicationBase.cpp | 162 ------------------------ src/Client/ClientBaseOptimizedParts.cpp | 4 +- 2 files changed, 2 insertions(+), 164 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 0306468d084..4aa8b6c0bbe 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -102,168 +102,6 @@ void ClientApplicationBase::setupSignalHandler() throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); } - -namespace -{ - -/// Define transparent hash to we can use -/// std::string_view with the containers -struct TransparentStringHash -{ - using is_transparent = void; - size_t operator()(std::string_view txt) const - { - return std::hash{}(txt); - } -}; - -/* - * This functor is used to parse command line arguments and replace dashes with underscores, - * allowing options to be specified using either dashes or underscores. - */ -class OptionsAliasParser -{ -public: - explicit OptionsAliasParser(const boost::program_options::options_description& options) - { - options_names.reserve(options.options().size()); - for (const auto& option : options.options()) - options_names.insert(option->long_name()); - } - - /* - * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options - * Implements boost::program_options::ext_parser logic - */ - std::pair operator()(const std::string & token) const - { - if (!token.starts_with("--")) - return {}; - std::string arg = token.substr(2); - - // divide token by '=' to separate key and value if options style=long_allow_adjacent - auto pos_eq = arg.find('='); - std::string key = arg.substr(0, pos_eq); - - if (options_names.contains(key)) - // option does not require any changes, because it is already correct - return {}; - - std::replace(key.begin(), key.end(), '-', '_'); - if (!options_names.contains(key)) - // after replacing '-' with '_' argument is still unknown - return {}; - - std::string value; - if (pos_eq != std::string::npos && pos_eq < arg.size()) - value = arg.substr(pos_eq + 1); - - return {key, value}; - } - -private: - std::unordered_set options_names; -}; - -} - -/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests -#if defined(__clang__) -#pragma clang optimize on -#endif -void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) -{ - if (allow_repeated_settings) - addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); - else - addProgramOptions(cmd_settings, options_description.main_description.value()); - - if (allow_merge_tree_settings) - { - /// Add merge tree settings manually, because names of some settings - /// may clash. Query settings have higher priority and we just - /// skip ambiguous merge tree settings. - auto & main_options = options_description.main_description.value(); - - std::unordered_set> main_option_names; - for (const auto & option : main_options.options()) - main_option_names.insert(option->long_name()); - - for (const auto & setting : cmd_merge_tree_settings.all()) - { - const auto add_setting = [&](const std::string_view name) - { - if (auto it = main_option_names.find(name); it != main_option_names.end()) - return; - - if (allow_repeated_settings) - addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); - else - addProgramOption(cmd_merge_tree_settings, main_options, name, setting); - }; - - const auto & setting_name = setting.getName(); - - add_setting(setting_name); - - const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); - if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) - { - for (const auto alias : it->second) - { - add_setting(alias); - } - } - } - } - - /// Parse main commandline options. - auto parser = po::command_line_parser(arguments) - .options(options_description.main_description.value()) - .extra_parser(OptionsAliasParser(options_description.main_description.value())) - .allow_unregistered(); - po::parsed_options parsed = parser.run(); - - /// Check unrecognized options without positional options. - auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); - if (!unrecognized_options.empty()) - { - auto hints = this->getHints(unrecognized_options[0]); - if (!hints.empty()) - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", - unrecognized_options[0], toString(hints)); - - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); - } - - /// Check positional options. - for (const auto & op : parsed.options) - { - if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") - && !op.original_tokens[0].empty() && !op.value.empty()) - { - /// Two special cases for better usability: - /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" - /// These are relevant for interactive usage - user-friendly, but questionable in general. - /// In case of ambiguity or for scripts, prefer using proper options. - - const auto & token = op.original_tokens[0]; - po::variable_value value(boost::any(op.value), false); - - const char * option; - if (token.contains(' ')) - option = "query"; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - - if (!options.emplace(option, value).second) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - } - } - - po::store(parsed, options); -} - void ClientApplicationBase::addMultiquery(std::string_view query, Arguments & common_arguments) const { common_arguments.emplace_back("--multiquery"); diff --git a/src/Client/ClientBaseOptimizedParts.cpp b/src/Client/ClientBaseOptimizedParts.cpp index 421843a0e79..297b8e7ce51 100644 --- a/src/Client/ClientBaseOptimizedParts.cpp +++ b/src/Client/ClientBaseOptimizedParts.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -80,7 +80,7 @@ private: } -void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) { if (allow_repeated_settings) addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); From 6f16ca02a74a81956b7524958c5f94eb7a2c7bf5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 16:14:31 +0000 Subject: [PATCH 185/496] Fixed Style Check --- src/Client/ClientApplicationBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 4aa8b6c0bbe..9f133616d2e 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -31,7 +31,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int CANNOT_SET_SIGNAL_HANDLER; - extern const int UNRECOGNIZED_ARGUMENTS; } static ClientInfo::QueryKind parseQueryKind(const String & query_kind) From 75728ac56d83b85e476162a745686837cb194b73 Mon Sep 17 00:00:00 2001 From: Halersson Paris <142428374+halersson@users.noreply.github.com> Date: Mon, 29 Jul 2024 14:42:58 -0300 Subject: [PATCH 186/496] Fix typo --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index e837d4d5e20..bc5e8292192 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -420,7 +420,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() int num_row_groups = metadata->num_row_groups(); row_group_batches.reserve(num_row_groups); - auto adative_chunk_size = [&](int row_group_idx) -> size_t + auto adaptive_chunk_size = [&](int row_group_idx) -> size_t { size_t total_size = 0; auto row_group_meta = metadata->RowGroup(row_group_idx); @@ -457,7 +457,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().row_groups_idxs.push_back(row_group); row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); - auto rows = adative_chunk_size(row_group); + auto rows = adaptive_chunk_size(row_group); row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; } } From 06b3185e57953c43531e4281f22bc03a8cd424e7 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 29 Jul 2024 20:05:29 +0200 Subject: [PATCH 187/496] fixes --- src/Interpreters/InterpreterCreateQuery.cpp | 14 +- .../test_restore_external_engines/test.py | 133 ++++++++++++++---- 2 files changed, 111 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a5eb3a83365..6e689c59c09 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1031,13 +1031,6 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*to_engine, getContext()->getSettingsRef().default_table_engine.value); } - /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to - /// Null table engine. - else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) - { - if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) - setNullTableEngine(*create.storage); - } return; } } @@ -1050,6 +1043,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); } + /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to + /// Null table engine. + else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) + { + if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) + setNullTableEngine(*create.storage); + } return; } diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py index eb88da6b61f..cf189f2a6ed 100644 --- a/tests/integration/test_restore_external_engines/test.py +++ b/tests/integration/test_restore_external_engines/test.py @@ -7,52 +7,75 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] -node1 = cluster.add_instance("replica1", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) -node2 = cluster.add_instance("replica2", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) -node3 = cluster.add_instance("replica3", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node1 = cluster.add_instance( + "replica1", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node2 = cluster.add_instance( + "replica2", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node3 = cluster.add_instance( + "replica3", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) nodes = [node1, node2, node3] backup_id_counter = 0 + def new_backup_name(): global backup_id_counter backup_id_counter += 1 return f"Disk('backups', '{backup_id_counter}/')" + def cleanup_nodes(nodes, dbname): for node in nodes: node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") + def fill_nodes(nodes, dbname): cleanup_nodes(nodes, dbname) for node in nodes: - node.query(f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')") + node.query( + f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')" + ) + def drop_mysql_table(conn, tableName): with conn.cursor() as cursor: cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") + def get_mysql_conn(cluster): conn = pymysql.connect( - user="root", password="clickhouse", host=cluster.mysql8_ip, port=cluster.mysql8_port + user="root", + password="clickhouse", + host=cluster.mysql8_ip, + port=cluster.mysql8_port, ) return conn + def fill_tables(cluster, dbname): fill_nodes(nodes, dbname) conn = get_mysql_conn(cluster) with conn.cursor() as cursor: - cursor.execute( - "DROP DATABASE IF EXISTS clickhouse" - ) - cursor.execute( - "CREATE DATABASE clickhouse" - ) - cursor.execute( - "DROP TABLE IF EXISTS clickhouse.inference_table" - ) + cursor.execute("DROP DATABASE IF EXISTS clickhouse") + cursor.execute("CREATE DATABASE clickhouse") + cursor.execute("DROP TABLE IF EXISTS clickhouse.inference_table") cursor.execute( "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" ) @@ -66,17 +89,30 @@ def fill_tables(cluster, dbname): node1.query( f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" ) - node1.query(f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})") + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})" + ) node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" - assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") == expected - assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") == expected + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") + == expected + ) + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") + == expected + ) assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" - assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" - assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") == "100\n" + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" + ) + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") + == "100\n" + ) assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" @@ -92,6 +128,7 @@ def start_cluster(): finally: cluster.shutdown() + def test_restore_table(start_cluster): fill_tables(cluster, "replicated") backup_name = new_backup_name() @@ -107,12 +144,26 @@ def test_restore_table(start_cluster): assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true") + node3.query( + f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true" + ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine") == "1\t100\n" - assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function") == "1\t100\n" - assert node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" + ) + == "1\t100\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function" + ) + == "1\t100\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + ) cleanup_nodes(nodes, "replicated") @@ -132,12 +183,36 @@ def test_restore_table_null(start_cluster): assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1") + node3.query( + f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1" + ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" - assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function") == "0\t0\n" - assert node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" - assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'") == "Null\nNull\n" - assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'") == "MergeTree\n" + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" + ) + == "0\t0\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function" + ) + == "0\t0\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'" + ) + == "Null\nNull\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'" + ) + == "MergeTree\n" + ) cleanup_nodes(nodes, "replicated2") From cb056cf3a5080cbff61f6efd070733ae2061d5b8 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Mon, 29 Jul 2024 12:05:44 -0700 Subject: [PATCH 188/496] Add camelCase aliases for percentRank() and denseRank() for percent_rank() and dense_rank() --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 2b255c5120e..a1b46c8e36c 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2721,20 +2721,24 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("dense_rank", {[](const std::string & name, + factory.registerFunction("denseRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("percent_rank", {[](const std::string & name, + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + + factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { From 9811a2e71b825a55c376edfb38303c817493cd9e Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 28 Jul 2024 17:32:32 -0700 Subject: [PATCH 189/496] Add test 03213_denseRank_percentRank_alias --- ...3213_denseRank_percentRank_alias.reference | 45 ++++++++++++++ .../03213_denseRank_percentRank_alias.sql | 59 +++++++++++++++++++ 2 files changed, 104 insertions(+) create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference new file mode 100644 index 00000000000..b49e179df68 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference @@ -0,0 +1,45 @@ +---- denseRank() ---- +0 0 0 1 1 1 1 +3 0 0 2 2 2 2 +1 0 1 3 3 3 3 +4 0 1 4 4 4 4 +2 0 2 5 5 5 5 +6 1 0 1 1 1 1 +9 1 0 2 2 2 2 +7 1 1 3 3 3 3 +5 1 2 4 4 4 4 +8 1 2 5 5 5 5 +12 2 0 1 1 1 1 +10 2 1 2 2 2 2 +13 2 1 3 3 3 3 +11 2 2 4 4 4 4 +14 2 2 5 5 5 5 +15 3 0 1 1 1 1 +18 3 0 2 2 2 2 +16 3 1 3 3 3 3 +19 3 1 4 4 4 4 +17 3 2 5 5 5 5 +21 4 0 1 1 1 1 +24 4 0 2 2 2 2 +22 4 1 3 3 3 3 +20 4 2 4 4 4 4 +23 4 2 5 5 5 5 +27 5 0 1 1 1 1 +25 5 1 2 2 2 2 +28 5 1 3 3 3 3 +26 5 2 4 4 4 4 +29 5 2 5 5 5 5 +30 6 0 1 1 1 1 +---- percentRank() ---- +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 +Others Unknow 200 1 0 diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql new file mode 100644 index 00000000000..ff841294eb1 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql @@ -0,0 +1,59 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/67042 +-- Reference generated using percent_rank() and dense_rank() + +-- From ClickHouse/tests/queries/0_stateless/01591_window_functions.sql (for deterministic query) +SELECT '---- denseRank() ----'; +select number, p, o, + count(*) over w, + rank() over w, + denseRank() over w, + row_number() over w +from (select number, intDiv(number, 5) p, mod(number, 3) o + from numbers(31) order by o, number) t +window w as (partition by p order by o, number) +order by p, o, number +settings max_block_size = 2; + +-- Modifed from ClickHouse/tests/queries/0_stateless/01592_window_functions.sql (for deterministic query) +SELECT '---- percentRank() ----'; + +drop table if exists product_groups; +drop table if exists products; + +CREATE TABLE product_groups ( + group_id Int64, + group_name String +) Engine = Memory; + +CREATE TABLE products ( + product_id Int64, + product_name String, + price DECIMAL(11, 2), + group_id Int64 +) Engine = Memory; + +INSERT INTO product_groups VALUES (1, 'Smartphone'),(2, 'Laptop'),(3, 'Tablet'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (1, 'Microsoft Lumia', 1, 200), (2, 'HTC One', 1, 400), (3, 'Nexus', 1, 500), (4, 'iPhone', 1, 900),(5, 'HP Elite', 2, 1200),(6, 'Lenovo Thinkpad', 2, 700),(7, 'Sony VAIO', 2, 700),(8, 'Dell Vostro', 2, 800),(9, 'iPad', 3, 700),(10, 'Kindle Fire', 3, 150),(11, 'Samsung Galaxy Tab', 3, 200); +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + + +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percentRank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + price ASC, + product_name ASC; + +drop table product_groups; +drop table products; From 3df2d88cf13ad552058a6958630741d7cdab9d3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 21:09:11 +0200 Subject: [PATCH 190/496] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 07b37835dda..620b7c99bac 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,7 +45,6 @@ * Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). #### Performance Improvement -* Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). * Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). * Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). * Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). From 756bde1158c4b3e6e65d324436291d53b9e25fbb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:27:15 +0100 Subject: [PATCH 191/496] rm file --- tests/performance/views_max_insert_threads.xml | 11 ----------- 1 file changed, 11 deletions(-) delete mode 100644 tests/performance/views_max_insert_threads.xml diff --git a/tests/performance/views_max_insert_threads.xml b/tests/performance/views_max_insert_threads.xml deleted file mode 100644 index 473bcd02ab8..00000000000 --- a/tests/performance/views_max_insert_threads.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - - - - - - From 2308a362a0aca716e2e50d3eb5283bdfd575e023 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 29 Jul 2024 19:31:46 +0000 Subject: [PATCH 192/496] Disable 02932_refreshable_materialized_views --- .../02932_refreshable_materialized_views.sh | 304 +----------------- 1 file changed, 1 insertion(+), 303 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 9081035579d..6df3c391ddb 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -1,306 +1,4 @@ #!/usr/bin/env bash # Tags: atomic-database -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh -# scheduling is done in UTC. -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" - -$CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" - - -# Basic refreshing. -$CLICKHOUSE_CLIENT -nq " - create materialized view a - refresh after 2 second - engine Memory - empty - as select number as x from numbers(2) union all select rand64() as x" -$CLICKHOUSE_CLIENT -nq "select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes"; -$CLICKHOUSE_CLIENT -nq "show create a" -# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] -do - sleep 0.1 -done -start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" -# Check table contents. -$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" -# Wait for table contents to change. -res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" -while : -do - res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res2" == "$res1" ] || break - sleep 0.1 -done -# Wait for another change. -while : -do - res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res3" == "$res2" ] || break - sleep 0.1 -done -# Check that the two changes were at least 1 second apart, in particular that we're not refreshing -# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer -# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. -$CLICKHOUSE_CLIENT -nq " - select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); - select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" - -# Create a source table from which views will read. -$CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1" - -# Switch to fake clock, change refresh schedule, change query. -$CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table a modify refresh every 2 year; - alter table a modify query select x*2 as x from src; - select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; - show create a;" -# Advance time to trigger the refresh. -$CLICKHOUSE_CLIENT -nq " - select '<5: no refresh>', count() from a; - system test view a set fake time '2052-02-03 04:05:06';" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<6: refreshed>', * from a; - select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" - -# Create a dependent view, refresh it once. -$CLICKHOUSE_CLIENT -nq " - create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; - show create b; - system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2052-11-11 11:11:11' ] -do - sleep 0.1 -done -# Next refresh shouldn't start until the dependency refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<8: refreshed>', * from b; - select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; - system test view b set fake time '2054-01-24 23:22:21';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] -do - sleep 0.1 -done -# Update source table (by dropping and re-creating it - to test that tables are looked up by name -# rather than uuid), kick off refresh of the dependency. -$CLICKHOUSE_CLIENT -nq " - select '<10: waiting>', view, status, remaining_dependencies, next_refresh_time from refreshes; - drop table src; - create table src (x Int16) engine Memory as select 2; - system test view a set fake time '2054-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] -do - sleep 0.1 -done -# Both tables should've refreshed. -$CLICKHOUSE_CLIENT -nq " - select '<11: chain-refreshed a>', * from a; - select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception from refreshes;" - -# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to -# catch up to the same cycle. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2059-01-01 00:00:00'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2061-01-01 00:00:00'; - system test view a set fake time '2057-01-01 00:00:00';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] -do - sleep 0.1 -done - -$CLICKHOUSE_CLIENT -nq " - select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; - truncate src; - insert into src values (3); - system test view a set fake time '2060-02-02 02:02:02';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<15: chain-refreshed a>', * from a; - select '<16: chain-refreshed b>', * from b; - select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" - -# Get to WaitingForDependencies state and remove the depencency. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2062-03-03 03:03:03'" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table b modify refresh every 2 year" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; - show create b;" - -# Select from a table that doesn't exist, get an exception. -$CLICKHOUSE_CLIENT -nq " - drop table a; - drop table b; - create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; - drop table src;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'c' -- $LINENO" | xargs`" != 'Exception' ] -do - sleep 0.1 -done -# Check exception, create src, expect successful refresh. -$CLICKHOUSE_CLIENT -nq " - select '<19: exception>', exception ilike '%UNKNOWN_TABLE%' ? '1' : exception from refreshes where view = 'c'; - create table src (x Int64) engine Memory as select 1; - system refresh view c;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -# Rename table. -$CLICKHOUSE_CLIENT -nq " - select '<20: unexception>', * from c; - rename table c to d; - select '<21: rename>', * from d; - select '<22: rename>', view, last_refresh_result from refreshes;" - -# Do various things during a refresh. -# First make a nonempty view. -$CLICKHOUSE_CLIENT -nq " - drop table d; - truncate src; - insert into src values (1) - create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -# Stop refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<23: simple refresh>', * from e; - system stop view e;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Disabled' ] -do - sleep 0.1 -done -# Make refreshes slow, wait for a slow refresh to start. (We stopped refreshes first to make sure -# we wait for a slow refresh, not a previous fast one.) -$CLICKHOUSE_CLIENT -nq " - insert into src select * from numbers(1000) settings max_block_size=1; - system start view e;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] -do - sleep 0.1 -done -# Rename. -$CLICKHOUSE_CLIENT -nq " - rename table e to f; - select '<24: rename during refresh>', * from f; - select '<25: rename during refresh>', view, status from refreshes where view = 'f'; - alter table f modify refresh after 10 year;" - -# Cancel. -$CLICKHOUSE_CLIENT -nq " - system cancel view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Cancelled' ] -do - sleep 0.1 -done - -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" = 'Running' ] -do - sleep 0.1 -done - -# Check that another refresh doesn't immediately start after the cancelled one. -$CLICKHOUSE_CLIENT -nq " - select '<27: cancelled>', view, status from refreshes where view = 'f'; - system refresh view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Running' ] -do - sleep 0.1 -done -# Drop. -$CLICKHOUSE_CLIENT -nq " - drop table f; - select '<28: drop during refresh>', view, status from refreshes;" - -# Try OFFSET and RANDOMIZE FOR. -$CLICKHOUSE_CLIENT -nq " - create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42; - show create g; - system test view g set fake time '2050-02-03 15:30:13';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - with '2050-02-10 04:00:00'::DateTime as expected - select '<29: randomize>', abs(next_refresh_time::Int64 - expected::Int64) <= 3600*(24*4+1), next_refresh_time != expected from refreshes;" - -# Send data 'TO' an existing table. -$CLICKHOUSE_CLIENT -nq " - drop table g; - create table dest (x Int64) engine MergeTree order by x; - truncate src; - insert into src values (1); - create materialized view h refresh every 1 second to dest empty as select x*10 as x from src; - show create h;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<30: to existing table>', * from dest; - insert into src values (2);" -while [ "`$CLICKHOUSE_CLIENT -nq "select count() from dest -- $LINENO" | xargs`" != '2' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<31: to existing table>', * from dest; - drop table dest; - drop table src; - drop table h;" - -# EMPTY -$CLICKHOUSE_CLIENT -nq " - create materialized view i refresh after 1 year engine Memory empty as select number as x from numbers(2); - create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2)" -while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<32: empty>', view, status, last_refresh_result from refreshes order by view; - drop table i; - drop table j" - -$CLICKHOUSE_CLIENT -nq " - drop table refreshes;" +# TODO: Re-add this test in https://github.com/ClickHouse/ClickHouse/pull/58934 From f3d5859c8bd5d1fb43f0d636d6cff3062e4ca267 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 19:35:37 +0000 Subject: [PATCH 193/496] Fix new test --- ...dynamic_read_subcolumns_small.reference.j2 | 192 +++++++++--------- ...03036_dynamic_read_subcolumns_small.sql.j2 | 16 +- 2 files changed, 104 insertions(+), 104 deletions(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 index be3f4e53990..d6add681f51 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 @@ -25,26 +25,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -115,6 +115,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -134,7 +135,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -295,26 +295,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -475,26 +475,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -655,7 +655,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -735,6 +734,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] Array(Array(Dynamic)) Array(Variant(String, UInt64)) None @@ -762,26 +762,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -852,6 +852,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -871,7 +872,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -1032,26 +1032,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -1212,26 +1212,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -1392,7 +1392,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -1472,6 +1471,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] Array(Array(Dynamic)) Array(Variant(String, UInt64)) None @@ -1499,26 +1499,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -1589,6 +1589,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -1608,7 +1609,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -1769,26 +1769,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -1949,26 +1949,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -2129,7 +2129,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -2209,3 +2208,4 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 index 1ed836fbeee..3253d7a6c68 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 @@ -29,14 +29,14 @@ select count() from test where not empty(d.`Array(Array(Dynamic))`); select count() from test where d is NULL; select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); -select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id; -select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id; -select d.Int8, d.Date, d.`Array(String)` from test order by id; -select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id; -select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id; +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.Int8, d.Date, d.`Array(String)` from test order by id, d; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id, d; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id, d; drop table test; From 8dfe4a93f6c1afde8475984a899cd5604f415d78 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 29 Jul 2024 22:07:10 +0200 Subject: [PATCH 194/496] Rewrite function get_broken_projections_info() without using system.errors --- .../test_broken_projections/test.py | 86 +++++++++---------- 1 file changed, 42 insertions(+), 44 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 2cbbee5563f..9493937d936 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -148,23 +148,22 @@ def break_part(node, table, part): bash(node, f"rm '{part_path}/columns.txt'") -def get_broken_projections_info(node, table, active=True): +def get_broken_projections_info(node, table, part=None, projection=None, active=True): + parent_name_filter = f" AND parent_name = '{part}'" if part else "" + name_filter = f" AND name = '{projection}'" if projection else "" return node.query( f""" - SELECT parent_name, name, errors.name FROM - ( - SELECT parent_name, name, exception_code + SELECT parent_name, name, exception FROM system.projection_parts WHERE table='{table}' AND database=currentDatabase() AND is_broken = 1 AND active = {active} - ) AS parts_info - INNER JOIN system.errors AS errors - ON parts_info.exception_code = errors.code + {parent_name_filter} + {name_filter} ORDER BY parent_name, name """ - ).strip() + ) def get_projections_info(node, table): @@ -312,8 +311,8 @@ def test_broken_ignored(cluster): # Projection 'proj1' from part all_2_2_0 will now appear in broken parts info # because it was marked broken during "check table" query. - assert "all_2_2_0\tproj1\tFILE_DOESNT_EXIST" in get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj1" ) # Check table query will also show a list of parts which have broken projections. @@ -323,14 +322,14 @@ def test_broken_ignored(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0 will now appear in broken parts info. - assert "all_2_2_0\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) # Second select works, because projection is now marked as broken. @@ -340,7 +339,7 @@ def test_broken_ignored(cluster): break_projection(node, table_name, "proj2", "all_3_3_0", "data") # It will not yet appear in broken projections info. - assert "all_3_3_0" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, part="all_3_3_0") insert(node, table_name, 20, 5) insert(node, table_name, 25, 5) @@ -371,8 +370,8 @@ def test_broken_ignored(cluster): node, table_name ) - assert "all_3_3_0" in get_broken_projections_info(node, table_name, active=False) - assert "all_2_2_0" in get_broken_projections_info(node, table_name, active=True) + assert get_broken_projections_info(node, table_name, part="all_3_3_0", active=False) + assert get_broken_projections_info(node, table_name, part="all_2_2_0", active=True) # 0 because of all_2_2_0 check(node, table_name, 0) @@ -396,8 +395,8 @@ def test_materialize_broken_projection(cluster): break_projection(node, table_name, "proj1", "all_1_1_0", "metadata") reattach(node, table_name) - assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" ) assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( node, table_name @@ -406,8 +405,8 @@ def test_materialize_broken_projection(cluster): break_projection(node, table_name, "proj2", "all_1_1_0", "data") reattach(node, table_name) - assert "all_1_1_0\tproj2\tFILE_DOESNT_EXIST" in get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj2" ) assert "Part all_1_1_0 has a broken projection proj2" in check_table_full( node, table_name @@ -469,8 +468,8 @@ def test_broken_projections_in_backups_2(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "part") check(node, table_name, 0, "proj2", "ErrnoException") - assert "all_2_2_0\tproj2\tFILE_DOESNT_EXIST" == get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) assert "FILE_DOESNT_EXIST" in node.query_and_get_error( @@ -524,8 +523,8 @@ def test_broken_projections_in_backups_3(cluster): assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( node, table_name ) - assert "all_1_1_0\tproj1\tFILE_DOESNT_EXIST" == get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" ) backup_name = f"b4-{get_random_string()}" @@ -545,8 +544,11 @@ def test_broken_projections_in_backups_3(cluster): ) check(node, table_name, 0) - assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( - node, table_name + assert ( + "Projection directory proj1.proj does not exist while loading projections" + in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" + ) ) @@ -569,7 +571,7 @@ def test_check_part_thread(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST", do_check_command=False) @@ -606,15 +608,15 @@ def test_broken_on_start(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" # We will mark projection as broken. check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0 will now appear in broken parts info. - assert "all_2_2_0\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) # Second select works, because projection is now marked as broken. @@ -623,7 +625,7 @@ def test_broken_on_start(cluster): node.restart_clickhouse() # It will not yet appear in broken projections info. - assert "proj2" in get_broken_projections_info(node, table_name) + assert get_broken_projections_info(node, table_name, projection="proj2") # Select works check(node, table_name, 0) @@ -654,7 +656,7 @@ def test_mutation_with_broken_projection(cluster): node, table_name ) - assert "" == get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name) check(node, table_name, 1) @@ -662,21 +664,21 @@ def test_mutation_with_broken_projection(cluster): break_projection(node, table_name, "proj2", "all_2_2_0_4", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" # We will mark projection as broken. check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0_4 will now appear in broken parts info. - assert "all_2_2_0_4\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0_4", projection="proj2" ) # Second select works, because projection is now marked as broken. check(node, table_name, 0) - assert "all_2_2_0_4" in get_broken_projections_info(node, table_name) + assert get_broken_projections_info(node, table_name, part="all_2_2_0_4") node.query( f"ALTER TABLE {table_name} DELETE WHERE _part == 'all_0_0_0_4' SETTINGS mutations_sync = 1" @@ -690,14 +692,10 @@ def test_mutation_with_broken_projection(cluster): # Still broken because it was hardlinked. broken = get_broken_projections_info(node, table_name) - assert ( - "all_2_2_0_5" in broken or "" == broken - ) # second could be because of a merge. + if broken: # can be not broken because of a merge. + assert get_broken_projections_info(node, table_name, part="all_2_2_0_5") - if "" == broken: - check(node, table_name, 1) - else: - check(node, table_name, 0) + check(node, table_name, not broken) node.query( f"ALTER TABLE {table_name} DELETE WHERE c == 13 SETTINGS mutations_sync = 1" @@ -710,6 +708,6 @@ def test_mutation_with_broken_projection(cluster): ) # Not broken anymore. - assert "" == get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name) check(node, table_name, 1) From f94bebb0530b7a9fdd3db104ad4261a467fafad3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 29 Jul 2024 22:36:59 +0000 Subject: [PATCH 195/496] fix --- docs/en/sql-reference/window-functions/lagInFrame.md | 2 +- docs/en/sql-reference/window-functions/leadInFrame.md | 2 +- src/Processors/Transforms/WindowTransform.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index 049e095c10f..de6e9005baa 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -23,7 +23,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Parameters** - `x` — Column name. - `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). -- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - default value of column type when omitted). **Returned value** diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index fc1b92cc266..4a82c03f6e6 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -23,7 +23,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Parameters** - `x` — Column name. - `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). -- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - default value of column type when omitted). **Returned value** diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1eac08780e9..f76e2d64368 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2408,7 +2408,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types[0]->equals(*argument_types[2])) return; - const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); + const auto supertype = tryGetLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); if (!supertype) { throw Exception(ErrorCodes::BAD_ARGUMENTS, From 2aafd711463d30cb7803a054f434268335817db8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 19:08:07 +0200 Subject: [PATCH 196/496] Moved settings to ServerSettings and made the table drop even faster (cherry picked from commit e1eb542dcc2b9fbc6a470a3cd9a183e79c86d7c7) --- programs/local/LocalServer.cpp | 5 +++ programs/server/Server.cpp | 5 +++ src/Core/ServerSettings.cpp | 2 +- src/Core/ServerSettings.h | 9 ++++ src/IO/SharedThreadPools.cpp | 10 +++++ src/IO/SharedThreadPools.h | 3 ++ src/Interpreters/DatabaseCatalog.cpp | 50 ++++++++++------------- src/Interpreters/DatabaseCatalog.h | 15 ------- src/Interpreters/InterpreterDropQuery.cpp | 19 +++++++-- 9 files changed, 70 insertions(+), 48 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 88d5a0253d1..250c5e3b6c8 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -184,6 +184,11 @@ void LocalServer::initialize(Poco::Util::Application & self) cleanup_threads, 0, // We don't need any threads one all the parts will be deleted cleanup_threads); + + getDatabaseCatalogDropTablesThreadPool().initialize( + server_settings.database_catalog_drop_table_concurrency, + 0, // We don't need any threads if there are no DROP queries. + server_settings.database_catalog_drop_table_concurrency); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..dd56114de0f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1035,6 +1035,11 @@ try 0, // We don't need any threads once all the tables will be created max_database_replicated_create_table_thread_pool_size); + getDatabaseCatalogDropTablesThreadPool().initialize( + server_settings.database_catalog_drop_table_concurrency, + 0, // We don't need any threads if there are no DROP queries. + server_settings.database_catalog_drop_table_concurrency); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index fbf86d3e9ad..6c498014996 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1,4 +1,4 @@ -#include "ServerSettings.h" +#include #include namespace DB diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..f2f78f70e91 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -66,6 +66,15 @@ namespace DB M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \ \ + /* Database Catalog */ \ + M(UInt64, database_atomic_delay_before_drop_table_sec, 8 * 60, "The delay during which a dropped table can be restored using the UNDROP statement. If DROP TABLE ran with a SYNC modifier, the setting is ignored.", 0) \ + M(UInt64, database_catalog_unused_dir_hide_timeout_sec, 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and this directory was not modified for last database_catalog_unused_dir_hide_timeout_sec seconds, the task will 'hide' this directory by removing all access rights. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'immediately'.", 0) \ + M(UInt64, database_catalog_unused_dir_rm_timeout_sec, 30 * 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and it was previously 'hidden' (see database_catalog_unused_dir_hide_timeout_sec) and this directory was not modified for last database_catalog_unused_dir_rm_timeout_sec seconds, the task will remove this directory. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'never'.", 0) \ + M(UInt64, database_catalog_unused_dir_cleanup_period_sec, 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. Sets scheduling period of the task. Zero means 'never'.", 0) \ + M(UInt64, database_catalog_drop_error_cooldown_sec, 5, "In case if drop table failed, ClickHouse will wait for this timeout before retrying the operation.", 0) \ + M(UInt64, database_catalog_drop_table_concurrency, 16, "The size of the threadpool used for dropping tables.", 0) \ + \ + \ M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index 3606ddd984c..cda7bc01bbf 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -23,6 +23,9 @@ namespace CurrentMetrics extern const Metric MergeTreeUnexpectedPartsLoaderThreads; extern const Metric MergeTreeUnexpectedPartsLoaderThreadsActive; extern const Metric MergeTreeUnexpectedPartsLoaderThreadsScheduled; + extern const Metric DatabaseCatalogThreads; + extern const Metric DatabaseCatalogThreadsActive; + extern const Metric DatabaseCatalogThreadsScheduled; extern const Metric DatabaseReplicatedCreateTablesThreads; extern const Metric DatabaseReplicatedCreateTablesThreadsActive; extern const Metric DatabaseReplicatedCreateTablesThreadsScheduled; @@ -166,4 +169,11 @@ StaticThreadPool & getDatabaseReplicatedCreateTablesThreadPool() return instance; } +/// ThreadPool used for dropping tables. +StaticThreadPool & getDatabaseCatalogDropTablesThreadPool() +{ + static StaticThreadPool instance("DropTablesThreadPool", CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, CurrentMetrics::DatabaseCatalogThreadsScheduled); + return instance; +} + } diff --git a/src/IO/SharedThreadPools.h b/src/IO/SharedThreadPools.h index 50adc70c9a0..06ccebd20b2 100644 --- a/src/IO/SharedThreadPools.h +++ b/src/IO/SharedThreadPools.h @@ -69,4 +69,7 @@ StaticThreadPool & getUnexpectedPartsLoadingThreadPool(); /// ThreadPool used for creating tables in DatabaseReplicated. StaticThreadPool & getDatabaseReplicatedCreateTablesThreadPool(); +/// ThreadPool used for dropping tables. +StaticThreadPool & getDatabaseCatalogDropTablesThreadPool(); + } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bb2dd158710..f64f8a06f38 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include #include #include @@ -189,13 +191,6 @@ StoragePtr TemporaryTableHolder::getTable() const void DatabaseCatalog::initializeAndLoadTemporaryDatabase() { - drop_delay_sec = getContext()->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec); - unused_dir_hide_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_hide_timeout_sec", unused_dir_hide_timeout_sec); - unused_dir_rm_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_rm_timeout_sec", unused_dir_rm_timeout_sec); - unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); - drop_error_cooldown_sec = getContext()->getConfigRef().getInt64("database_catalog_drop_error_cooldown_sec", drop_error_cooldown_sec); - drop_table_concurrency = getContext()->getConfigRef().getInt64("database_catalog_drop_table_concurrency", drop_table_concurrency); - auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); } @@ -203,7 +198,7 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase() void DatabaseCatalog::createBackgroundTasks() { /// It has to be done before databases are loaded (to avoid a race condition on initialization) - if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && unused_dir_cleanup_period_sec) + if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec) { auto cleanup_task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalogCleanupStoreDirectoryTask", [this]() { this->cleanupStoreDirectoryTask(); }); @@ -224,7 +219,7 @@ void DatabaseCatalog::startupBackgroundTasks() { (*cleanup_task)->activate(); /// Do not start task immediately on server startup, it's not urgent. - (*cleanup_task)->scheduleAfter(unused_dir_hide_timeout_sec * 1000); + (*cleanup_task)->scheduleAfter(static_cast(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec) * 1000); } (*drop_task)->activate(); @@ -1038,15 +1033,12 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() LOG_INFO(log, "Found {} partially dropped tables. Will load them and retry removal.", dropped_metadata.size()); - ThreadPool pool(CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, CurrentMetrics::DatabaseCatalogThreadsScheduled); + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); for (const auto & elem : dropped_metadata) { - pool.scheduleOrThrowOnError([&]() - { - this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first); - }); + runner([this, &elem](){ this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first); }); } - pool.wait(); + runner.waitForAllToFinishAndRethrowFirstError(); } String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const @@ -1135,7 +1127,13 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } else { - tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec}); + tables_marked_dropped.push_back + ({ + table_id, + table, + dropped_metadata_path, + drop_time + static_cast(getContext()->getServerSettings().database_atomic_delay_before_drop_table_sec) + }); if (first_async_drop_in_queue == tables_marked_dropped.end()) --first_async_drop_in_queue; } @@ -1289,13 +1287,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); for (const auto & item : tables_to_drop) { @@ -1332,7 +1324,7 @@ void DatabaseCatalog::dropTablesParallel(std::vectordrop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; + table_iterator->drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + getContext()->getServerSettings().database_catalog_drop_error_cooldown_sec; if (first_async_drop_in_queue == tables_marked_dropped.end()) --first_async_drop_in_queue; @@ -1342,7 +1334,7 @@ void DatabaseCatalog::dropTablesParallel(std::vectorscheduleAfter(unused_dir_cleanup_period_sec * 1000); + (*cleanup_task)->scheduleAfter(static_cast(getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec) * 1000); } bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir) @@ -1742,7 +1734,7 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP time_t current_time = time(nullptr); if (st.st_mode & (S_IRWXU | S_IRWXG | S_IRWXO)) { - if (current_time <= max_modification_time + unused_dir_hide_timeout_sec) + if (current_time <= max_modification_time + static_cast(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec)) return false; LOG_INFO(log, "Removing access rights for unused directory {} from disk {} (will remove it when timeout exceed)", unused_dir, disk_name); @@ -1758,6 +1750,8 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP } else { + auto unused_dir_rm_timeout_sec = static_cast(getContext()->getServerSettings().database_catalog_unused_dir_rm_timeout_sec); + if (!unused_dir_rm_timeout_sec) return false; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 23e38a6445e..83a302f117d 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -354,23 +354,8 @@ private: mutable std::mutex tables_marked_dropped_mutex; std::unique_ptr drop_task; - static constexpr time_t default_drop_delay_sec = 8 * 60; - time_t drop_delay_sec = default_drop_delay_sec; std::condition_variable wait_table_finally_dropped; - std::unique_ptr cleanup_task; - static constexpr time_t default_unused_dir_hide_timeout_sec = 60 * 60; /// 1 hour - time_t unused_dir_hide_timeout_sec = default_unused_dir_hide_timeout_sec; - static constexpr time_t default_unused_dir_rm_timeout_sec = 30 * 24 * 60 * 60; /// 30 days - time_t unused_dir_rm_timeout_sec = default_unused_dir_rm_timeout_sec; - static constexpr time_t default_unused_dir_cleanup_period_sec = 24 * 60 * 60; /// 1 day - time_t unused_dir_cleanup_period_sec = default_unused_dir_cleanup_period_sec; - - static constexpr time_t default_drop_error_cooldown_sec = 5; - time_t drop_error_cooldown_sec = default_drop_error_cooldown_sec; - - static constexpr size_t default_drop_table_concurrency = 10; - size_t drop_table_concurrency = default_drop_table_concurrency; std::unique_ptr reload_disks_task; std::mutex reload_disks_mutex; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index bad3e5277db..d8056ddd1a3 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -424,18 +425,28 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, auto table_context = Context::createCopy(getContext()); table_context->setInternalQuery(true); /// Do not hold extra shared pointers to tables - std::vector> tables_to_drop; + std::vector> tables_to_drop; // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { auto table_ptr = iterator->table(); - table_ptr->flushAndPrepareForShutdown(); - tables_to_drop.push_back({iterator->name(), table_ptr->isDictionary()}); + tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); } + /// Prepare tables for shutdown in parallel. + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); + for (const auto & [name, _] : tables_to_drop) + { + auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); + runner([my_table_ptr = std::move(table_ptr)](){ + my_table_ptr->flushAndPrepareForShutdown(); + }); + } + runner.waitForAllToFinishAndRethrowFirstError(); + for (const auto & table : tables_to_drop) { - query_for_table.setTable(table.first); + query_for_table.setTable(table.first.getTableName()); query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; From 1427b16689601d7dd29d26de99b233c132905fde Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 21:44:15 +0200 Subject: [PATCH 197/496] Fixed style --- src/Interpreters/DatabaseCatalog.cpp | 3 --- src/Interpreters/InterpreterDropQuery.cpp | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f64f8a06f38..30b151eb81d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -50,9 +50,6 @@ namespace CurrentMetrics { extern const Metric TablesToDropQueueSize; - extern const Metric DatabaseCatalogThreads; - extern const Metric DatabaseCatalogThreadsActive; - extern const Metric DatabaseCatalogThreadsScheduled; } namespace DB diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index d8056ddd1a3..ef560ec3405 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -438,7 +438,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, for (const auto & [name, _] : tables_to_drop) { auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); - runner([my_table_ptr = std::move(table_ptr)](){ + runner([my_table_ptr = std::move(table_ptr)]() + { my_table_ptr->flushAndPrepareForShutdown(); }); } From 1096a4ff33497c64eb786f6dbb603b18ccf804b1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 13:37:36 +0000 Subject: [PATCH 198/496] Fixed occasional LOGICAL_ERROR --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 30b151eb81d..eaf8cf1cc82 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1336,7 +1336,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector Date: Tue, 30 Jul 2024 00:39:16 +0200 Subject: [PATCH 199/496] Dont throw --- src/Interpreters/DatabaseCatalog.cpp | 30 ++++++++++------------------ 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index eaf8cf1cc82..98526e5c1cd 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1329,26 +1329,10 @@ void DatabaseCatalog::dropTablesParallel(std::vector Date: Mon, 29 Jul 2024 23:10:13 +0000 Subject: [PATCH 200/496] Fix Dwarf range list parsing in stack symbolizer --- src/Common/Dwarf.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 8439c01b22c..1f22e3b05aa 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -1559,8 +1559,7 @@ bool Dwarf::isAddrInRangeList(const CompilationUnit & cu, auto sp_start = addr_.substr(*cu.addr_base + index_start * sizeof(uint64_t)); auto start = read(sp_start); - auto sp_end = addr_.substr(*cu.addr_base + index_start * sizeof(uint64_t) + length); - auto end = read(sp_end); + auto end = start + length; if (start != end && address >= start && address < end) { return true; From 8f920d064ccca8ed8d9341b10d54ebef7500484c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 04:05:43 +0200 Subject: [PATCH 201/496] Fix inconsistent formatting of CODEC and STATISTICS --- src/Parsers/ASTFunction.cpp | 4 +++- src/Parsers/ASTFunction.h | 2 ++ src/Parsers/ExpressionElementParsers.cpp | 2 ++ src/Parsers/FunctionSecretArgumentsFinderAST.h | 4 +++- src/Parsers/IAST.h | 1 + src/Storages/StatisticsDescription.cpp | 1 + 6 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 230d4c778e8..cd9e910d45a 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -285,6 +285,8 @@ static bool formatNamedArgWithHiddenValue(IAST * arg, const IAST::FormatSettings void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.expression_list_prepend_whitespace = false; + if (kind == Kind::CODEC || kind == Kind::STATISTICS || kind == Kind::BACKUP_NAME) + frame.allow_operators = false; FormatStateStacked nested_need_parens = frame; FormatStateStacked nested_dont_need_parens = frame; nested_need_parens.need_parens = true; @@ -308,7 +310,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format /// Should this function to be written as operator? bool written = false; - if (arguments && !parameters && nulls_action == NullsAction::EMPTY) + if (arguments && !parameters && frame.allow_operators && nulls_action == NullsAction::EMPTY) { /// Unary prefix operators. if (arguments->children.size() == 1) diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index be2b6beae54..1b4a5928d1c 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -58,6 +58,8 @@ public: TABLE_ENGINE, DATABASE_ENGINE, BACKUP_NAME, + CODEC, + STATISTICS, }; Kind kind = Kind::ORDINARY_FUNCTION; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 865d07faaa7..9927acdcf17 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -696,6 +696,7 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto function_node = std::make_shared(); function_node->name = "CODEC"; + function_node->kind = ASTFunction::Kind::CODEC; function_node->arguments = expr_list_args; function_node->children.push_back(function_node->arguments); @@ -723,6 +724,7 @@ bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expect auto function_node = std::make_shared(); function_node->name = "STATISTICS"; + function_node->kind = ASTFunction::Kind::STATISTICS; function_node->arguments = stat_type; function_node->children.push_back(function_node->arguments); node = function_node; diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 5b77485afb0..94da30922cc 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -33,7 +33,9 @@ public: { case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; case ASTFunction::Kind::WINDOW_FUNCTION: break; - case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::CODEC: break; + case ASTFunction::Kind::STATISTICS: break; case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d70c1cd0b6c..e2cf7579667 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -256,6 +256,7 @@ public: bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) bool surround_each_list_element_with_parens = false; + bool allow_operators = true; /// Format some functions, such as "plus", "in", etc. as operators. size_t list_element_index = 0; const IAST * current_select = nullptr; }; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 9c5fd3604b2..63c849e3806 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -193,6 +193,7 @@ ASTPtr ColumnStatisticsDescription::getAST() const { auto function_node = std::make_shared(); function_node->name = "STATISTICS"; + function_node->kind = ASTFunction::Kind::STATISTICS; function_node->arguments = std::make_shared(); for (const auto & [type, desc] : types_to_desc) { From c1e7b7be89f5c5e39318ab093b00fa6cd8114ff3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 04:07:46 +0200 Subject: [PATCH 202/496] Add a test --- ...nconsistent_formatting_of_codecs_statistics.reference | 2 ++ ...03214_inconsistent_formatting_of_codecs_statistics.sh | 9 +++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference create mode 100755 tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh diff --git a/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference new file mode 100644 index 00000000000..7213baa3e5b --- /dev/null +++ b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference @@ -0,0 +1,2 @@ +ALTER TABLE t MODIFY COLUMN `c` CODEC(in(1, 2)) +ALTER TABLE t MODIFY COLUMN `c` STATISTICS(plus(1, 2)) diff --git a/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh new file mode 100755 index 00000000000..c3f8d89b9a4 --- /dev/null +++ b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Ensure that these (possibly incorrect) queries can at least be parsed back after formatting. +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t MODIFY COLUMN c CODEC(in(1, 2))" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t MODIFY COLUMN c STATISTICS(plus(1, 2))" | $CLICKHOUSE_FORMAT --oneline From cb6b6329c8e763f61f70797a95dab8ef24fd47d1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 30 Jul 2024 10:45:36 +0800 Subject: [PATCH 203/496] add session timezone settings --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 7e931e16e48..7d1da0c1579 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -8,5 +8,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" $CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" $CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone SETTINGS session_timezone='Asia/Shanghai'" $CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From dd5819ab6ab2df3231737a4808d0445ad5345555 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:16:09 +0200 Subject: [PATCH 204/496] Changelog sanity --- CHANGELOG.md | 55 +++++++++++++++++++++++++--------------------------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 620b7c99bac..06f7bcdd84e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,7 @@ * Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). * Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Function `tuple` will now try to construct named tuples in query (controlled by `enable_named_columns_in_function_tuple`). Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). +* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). #### New Feature * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). @@ -32,13 +33,14 @@ * Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). * Introduce `logger.console_log_level` server config to control the log level to the console (if enabled). [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). * Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). -* Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). +* Add `--memory-usage` option to client in non-interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). * Make an interactive client for clickhouse-disks, add local disk from the local directory. [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). * When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). +* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). #### Experimental Feature * Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). -* Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* Support rocksdb as backend storage of clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). * Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). * Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). * Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). @@ -46,8 +48,8 @@ #### Performance Improvement * Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). -* Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). -* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). +* Sizes of hash tables created by join (`parallel_hash` algorithm) are collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). * Improve performance of loading `plain_rewritable` metadata. [#65634](https://github.com/ClickHouse/ClickHouse/pull/65634) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Attaching tables on read-only disks will use fewer resources by not loading outdated parts. [#65635](https://github.com/ClickHouse/ClickHouse/pull/65635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). @@ -59,11 +61,11 @@ * DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). #### Improvement +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). * The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). -* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). -* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). +* Allow matching column names in a case-insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). @@ -71,7 +73,6 @@ * Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). * Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). -* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). * `arrayMin`/`arrayMax` can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). * Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). * Do not create format settings for each row when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). @@ -80,36 +81,35 @@ * Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new option to config `` which allow to specify how often clickhouse will reload config. [#65545](https://github.com/ClickHouse/ClickHouse/pull/65545) ([alesapin](https://github.com/alesapin)). * Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). -* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). * Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). * Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). * `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add a script to backup your files to ClickHouse. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* PostgreSQL source support cancel. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). -* Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* PostgreSQL source to support query cancellations. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). +* Make `allow_experimental_analyzer` be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). -* Allow to use `concat` function with empty arguments ``` sql :) select concat();. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). -* Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). -* Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). -* Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). -* Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). -* Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). -* Add settings to ignore ON CLUSTER clause in queries for named collection management with replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Allow to use `concat` function with empty arguments `:) select concat();`. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). +* Allow controlling named collections in `clickhouse-local`. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve Azure-related profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). +* Support ORC file read by writer's time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). +* Add settings to control connections to PostgreSQL. The setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. The setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). +* Reduce inaccuracy of `input_wait_elapsed_us`/`elapsed_us` in the `system.processors_profile_log`. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). +* Improve ProfileEvents for the filesystem cache. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). +* Add settings to ignore the `ON CLUSTER` clause in queries for named collection management with the replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Function `generateSnowflakeID` now allows to specify a machine ID as a parameter to prevent collisions in large clusters. [#66374](https://github.com/ClickHouse/ClickHouse/pull/66374) ([ZAWA_ll](https://github.com/Zawa-ll)). -* Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Disable suspending on `Ctrl+Z` in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add option for validating the primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). #### Bug Fix (user-visible misbehavior in an official stable release) -* Fix unexpected size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). * Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix unexpected sizes of `LowCardinality` columns in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). * Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). -* Fix the VALID UNTIL clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the `VALID UNTIL` clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix the remaining time column in `SHOW MERGES`. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). -* Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). -* Fix bug with cancellation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fixed crash while using `MaterializedMySQL` (which is an unsupported, experimental feature) with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). +* Fix logical error when `PREWHERE` expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug with the cancellation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). * Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). * Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). * Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). @@ -178,9 +178,6 @@ * Fix `indexHint` function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). * Fix AST formatting of 'create table b empty as a'. [#64951](https://github.com/ClickHouse/ClickHouse/pull/64951) ([Michael Kolupaev](https://github.com/al13n321)). -#### Build/Testing/Packaging Improvement -* Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). - ### ClickHouse release 24.6, 2024-07-01 #### Backward Incompatible Change From 3a7ffb3284003d853974baf12cf442bdc1105143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:19:10 +0200 Subject: [PATCH 205/496] Changelog sanity --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 06f7bcdd84e..9d1a63cb3a9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -156,7 +156,7 @@ * Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). * Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix error reporting while copying to S3 or AzureBlobStorage. [#66295](https://github.com/ClickHouse/ClickHouse/pull/66295) ([Vitaly Baranov](https://github.com/vitlibar)). -* Prevent watchdog from keeping descriptors of unlinked(rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Prevent watchdog from keeping descriptors of unlinked (rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). * Fix the bug that logicalexpressionoptimizerpass lost logical type of constant. [#66344](https://github.com/ClickHouse/ClickHouse/pull/66344) ([pn](https://github.com/chloro-pn)). * Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too aggressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. [#66402](https://github.com/ClickHouse/ClickHouse/pull/66402) ([vdimir](https://github.com/vdimir)). From 368b9a058379c2e7902fd9ee7a21b664f0500df9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:29:36 +0200 Subject: [PATCH 206/496] Changelog sanity --- CHANGELOG.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9d1a63cb3a9..722ae4f8268 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -39,11 +39,11 @@ * Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). #### Experimental Feature -* Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). -* Support rocksdb as backend storage of clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* Change binary serialization of the `Variant` data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). +* Support on-disk backend storage for clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). * Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). -* Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Support null map subcolumn for `Variant` and `Dynamic` subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading `Dynamic` subcolumns from altered `Memory` table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). * Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). #### Performance Improvement From f90b88c978d87225a5cf5f66136714ff5535d69c Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 30 Jul 2024 05:57:07 +0000 Subject: [PATCH 207/496] Reduced comprexity of the test --- ...2832_alter_max_sessions_for_user.reference | 7 +-- .../02832_alter_max_sessions_for_user.sh | 62 ++++++++----------- 2 files changed, 30 insertions(+), 39 deletions(-) diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference index f80f8738ff8..c2e103d61cb 100644 --- a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference @@ -1,8 +1,7 @@ -test_alter_profile case: max_session_count 1 alter_sessions_count 1 -test_alter_profile case: max_session_count 2 alter_sessions_count 1 +test_alter_profile case: max_sessions_for_user 1 +USER_SESSION_LIMIT_EXCEEDED +test_alter_profile case: max_sessions_for_user 2 USER_SESSION_LIMIT_EXCEEDED -test_alter_profile case: max_session_count 1 alter_sessions_count 2 -test_alter_profile case: max_session_count 2 alter_sessions_count 2 READONLY READONLY READONLY diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh index 87fbffdb1e6..55f9e3e97a4 100755 --- a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SESSION_ID_PREFIX="02832_alter_max_sessions_session_$$" QUERY_ID_PREFIX="02832_alter_max_sessions_query_$$" PROFILE="02832_alter_max_sessions_profile_$$" USER="02832_alter_max_sessions_user_$$" @@ -17,48 +16,41 @@ ${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" ${CLICKHOUSE_CLIENT} -q $"CREATE SETTINGS PROFILE ${PROFILE}" ${CLICKHOUSE_CLIENT} -q $"CREATE USER '${USER}' SETTINGS PROFILE '${PROFILE}'" -function run_sessions_set() +function wait_for_query_to_start() { - local sessions_count="$1" - local session_check="$2" - for ((i = 1 ; i <= ${sessions_count} ; i++)); do - local session_id="${SESSION_ID_PREFIX}_${i}" - local query_id="${QUERY_ID_PREFIX}_${i}" - # Write only expected error text - # More than alter_sessions_count queries will not start. - ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&query_id=${query_id}&session_id=${session_id}&session_check=${session_check}&session_timeout=600&function_sleep_max_microseconds_per_block=120000000" --data-binary "SELECT sleep(120)" | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' & + while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.2; done +} + +function test_alter_max_sessions_for_user() +{ + local max_sessions_for_user="$1" + echo $"test_alter_profile case: max_sessions_for_user ${max_sessions_for_user}" + + # Step 0: Set max_sessions_for_user. + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_sessions_for_user}" + + # Step 1: Simulaneously run `max_sessions_for_user` queries. These queries should run without any problems. + for ((i = 1 ; i <= max_sessions_for_user ; i++)); do + local query_id="${QUERY_ID_PREFIX}_${i}_${max_sessions_for_user}" + ${CLICKHOUSE_CLIENT} --max_block_size 1 --query_id $query_id --user $USER --function_sleep_max_microseconds_per_block=120000000 -q "SELECT sleepEachRow(0.1) FROM numbers(1200)" &>/dev/null & + wait_for_query_to_start $query_id done - for ((i = 1 ; i <= ${sessions_count} ; i++)); do - local query_id="${QUERY_ID_PREFIX}_${i}" - $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id' SYNC" >/dev/null + # Step 2: Run another `max_sessions_for_user` + 1 query. That query should fail. + local query_id="${QUERY_ID_PREFIX}_should_fail" + ${CLICKHOUSE_CLIENT} --query_id $query_id --user $USER -q "SELECT 1" 2>&1 | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' + + # Step 3: Stop running queries launched at step 1. + for ((i = 1 ; i <= max_sessions_for_user ; i++)); do + local query_id="${QUERY_ID_PREFIX}_${i}_${max_sessions_for_user}" + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id' ASYNC" >/dev/null done wait } -function test_alter_profile() -{ - local max_session_count="$1" - local alter_sessions_count="$2" - echo $"test_alter_profile case: max_session_count ${max_session_count} alter_sessions_count ${alter_sessions_count}" - - ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}" - - # Create sessions with $max_session_count restriction - run_sessions_set $max_session_count 0 - - # Update restriction to $alter_sessions_count - ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}" - - # Simultaneous sessions should use max settings from profile ($alter_sessions_count) - run_sessions_set $max_session_count 1 -} - -test_alter_profile 1 1 -test_alter_profile 2 1 -test_alter_profile 1 2 -test_alter_profile 2 2 +test_alter_max_sessions_for_user 1 +test_alter_max_sessions_for_user 2 ${CLICKHOUSE_CLIENT} -q "SELECT 1 SETTINGS max_sessions_for_user = 1" 2>&1 | grep -m 1 -o 'READONLY' | head -1 ${CLICKHOUSE_CLIENT} -q $"SET max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 From 7a53a14940ae1be299305548f0d024de7f279fe3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 09:29:16 +0200 Subject: [PATCH 208/496] Update 03213_deep_json.sql --- tests/queries/0_stateless/03213_deep_json.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03213_deep_json.sql b/tests/queries/0_stateless/03213_deep_json.sql index 4f79d99eb26..2a9476381ff 100644 --- a/tests/queries/0_stateless/03213_deep_json.sql +++ b/tests/queries/0_stateless/03213_deep_json.sql @@ -1,5 +1,5 @@ -- The default limit works. -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } -- Even if we relax the limit, it is also safe. SET input_format_json_max_depth = 100000; -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } From 6ab67323862391e520ac571f609f52b582248da5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 07:36:38 +0000 Subject: [PATCH 209/496] Delete .reference --- ...2_refreshable_materialized_views.reference | 44 ------------------- 1 file changed, 44 deletions(-) delete mode 100644 tests/queries/0_stateless/02932_refreshable_materialized_views.reference diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference deleted file mode 100644 index 2eb41590af1..00000000000 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ /dev/null @@ -1,44 +0,0 @@ -<1: created view> a [] 1 -CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x -<2: refreshed> 3 1 1 -<3: time difference at least> 1000 -<4: next refresh in> 2 -<4.5: altered> Scheduled Finished 2052-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src -<5: no refresh> 3 -<6: refreshed> 2 -<7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<8: refreshed> 20 -<9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 -<9: refreshed> b Scheduled Finished 2054-01-01 00:00:00 -<10: waiting> a Scheduled [] 2054-01-01 00:00:00 -<10: waiting> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 -<11: chain-refreshed a> 4 -<12: chain-refreshed b> 40 -<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 -<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 -<14: waiting for next cycle> a Scheduled [] 2058-01-01 00:00:00 -<14: waiting for next cycle> b WaitingForDependencies ['default.a'] 2060-01-01 00:00:00 -<15: chain-refreshed a> 6 -<16: chain-refreshed b> 60 -<17: chain-refreshed> a Scheduled 2062-01-01 00:00:00 -<17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 -<18: removed dependency> b Scheduled [] 2062-03-03 03:03:03 2064-01-01 00:00:00 5 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<19: exception> 1 -<20: unexception> 1 -<21: rename> 1 -<22: rename> d Finished -<23: simple refresh> 1 -<24: rename during refresh> 1 -<25: rename during refresh> f Running -<27: cancelled> f Scheduled -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 -<29: randomize> 1 1 -CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src -<30: to existing table> 10 -<31: to existing table> 10 -<31: to existing table> 20 -<32: empty> i Scheduled Unknown -<32: empty> j Scheduled Finished From 861bdb51f8e098c4b06cb14988e00febbb5a0ac7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 09:42:38 +0200 Subject: [PATCH 210/496] Fix test --- .../test_replicated_table_attach/test.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 02ef1ad6353..499220def2c 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -27,7 +27,20 @@ def started_cluster(): cluster.shutdown() +def start_clean_clickhouse(): + # remove fault injection if present + if "fault_injection.xml" in node.exec_in_container( + ["bash", "-c", "ls /etc/clickhouse-server/config.d"] + ): + print("Removing fault injection") + node.exec_in_container( + ["bash", "-c", "rm /etc/clickhouse-server/config.d/fault_injection.xml"] + ) + node.restart_clickhouse() + + def test_startup_with_small_bg_pool(started_cluster): + start_clean_clickhouse() node.query( "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" ) @@ -45,6 +58,7 @@ def test_startup_with_small_bg_pool(started_cluster): def test_startup_with_small_bg_pool_partitioned(started_cluster): + start_clean_clickhouse() node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" ) From a70571762f7d73a7ecc94981e8086418ecfdeb3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 10:08:35 +0200 Subject: [PATCH 211/496] Enable text_log by default --- programs/server/config.xml | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 94825a55f67..844aff8f668 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1130,8 +1130,7 @@ 7500 - + system part_log
@@ -1143,9 +1142,9 @@ false
- system text_log
@@ -1154,9 +1153,8 @@ 8192 524288 false - + trace
- --> From c427c4e2bba852f6f8f9b9346a9a2d0a09f0e4be Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 30 Jul 2024 10:34:47 +0200 Subject: [PATCH 212/496] Typo --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 98526e5c1cd..a8e5fd7e6aa 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1355,7 +1355,7 @@ void DatabaseCatalog::dropTableDataTask() } catch (...) { - /// We don't re-throw expection, because we are in a background pool. + /// We don't re-throw exception, because we are in a background pool. tryLogCurrentException(log, "Cannot drop tables. Will retry later."); } } From 3e6a1b99e023eb3d592c72c17ae4913a9074b5af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:05:45 +0200 Subject: [PATCH 213/496] Fix file/URI parsing with archive syntax --- src/IO/Archives/ArchiveUtils.cpp | 50 +++++++++++++++++++ src/IO/Archives/ArchiveUtils.h | 14 ++++++ src/IO/Archives/createArchiveReader.cpp | 13 ++--- src/IO/Archives/createArchiveWriter.cpp | 9 ++-- src/IO/S3/URI.cpp | 39 +++++++-------- src/IO/S3/URI.h | 3 +- src/Storages/StorageFile.cpp | 8 ++- src/TableFunctions/TableFunctionFile.cpp | 9 ++-- .../03214_parsing_archive_name_file.reference | 12 +++++ .../03214_parsing_archive_name_file.sh | 21 ++++++++ .../03215_parsing_archive_name_s3.reference | 2 + .../03215_parsing_archive_name_s3.sql | 6 +++ .../data_minio/::03215_archive.csv | 1 + .../data_minio/test :: 03215_archive.csv | 1 + .../data_minio/test::03215_archive.csv | 1 + 15 files changed, 145 insertions(+), 44 deletions(-) create mode 100644 src/IO/Archives/ArchiveUtils.cpp create mode 100644 tests/queries/0_stateless/03214_parsing_archive_name_file.reference create mode 100755 tests/queries/0_stateless/03214_parsing_archive_name_file.sh create mode 100644 tests/queries/0_stateless/03215_parsing_archive_name_s3.reference create mode 100644 tests/queries/0_stateless/03215_parsing_archive_name_s3.sql create mode 100644 tests/queries/0_stateless/data_minio/::03215_archive.csv create mode 100644 tests/queries/0_stateless/data_minio/test :: 03215_archive.csv create mode 100644 tests/queries/0_stateless/data_minio/test::03215_archive.csv diff --git a/src/IO/Archives/ArchiveUtils.cpp b/src/IO/Archives/ArchiveUtils.cpp new file mode 100644 index 00000000000..50009087de3 --- /dev/null +++ b/src/IO/Archives/ArchiveUtils.cpp @@ -0,0 +1,50 @@ +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +using namespace std::literals; +constexpr std::array tar_extensions{".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; +constexpr std::array zip_extensions{".zip"sv, ".zipx"sv}; +constexpr std::array sevenz_extensiosns{".7z"sv}; + +bool hasSupportedExtension(std::string_view path, const auto & supported_extensions) +{ + for (auto supported_extension : supported_extensions) + { + if (path.ends_with(supported_extension)) + return true; + } + + return false; +} + +} + +bool hasSupportedTarExtension(std::string_view path) +{ + return hasSupportedExtension(path, tar_extensions); +} + +bool hasSupportedZipExtension(std::string_view path) +{ + return hasSupportedExtension(path, zip_extensions); +} + +bool hasSupported7zExtension(std::string_view path) +{ + return hasSupportedExtension(path, sevenz_extensiosns); +} + +bool hasSupportedArchiveExtension(std::string_view path) +{ + return hasSupportedTarExtension(path) || hasSupportedZipExtension(path) || hasSupported7zExtension(path); +} + +} diff --git a/src/IO/Archives/ArchiveUtils.h b/src/IO/Archives/ArchiveUtils.h index 1b66be005a2..cdb731d1d57 100644 --- a/src/IO/Archives/ArchiveUtils.h +++ b/src/IO/Archives/ArchiveUtils.h @@ -10,3 +10,17 @@ #include #include #endif + +#include + +namespace DB +{ + +bool hasSupportedTarExtension(std::string_view path); +bool hasSupportedZipExtension(std::string_view path); +bool hasSupported7zExtension(std::string_view path); + +bool hasSupportedArchiveExtension(std::string_view path); + + +} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 782602091ac..dfa098eede0 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -12,7 +13,6 @@ extern const int CANNOT_UNPACK_ARCHIVE; extern const int SUPPORT_IS_DISABLED; } - std::shared_ptr createArchiveReader(const String & path_to_archive) { return createArchiveReader(path_to_archive, {}, 0); @@ -24,11 +24,7 @@ std::shared_ptr createArchiveReader( [[maybe_unused]] const std::function()> & archive_read_function, [[maybe_unused]] size_t archive_size) { - using namespace std::literals; - static constexpr std::array tar_extensions{ - ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; - - if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) + if (hasSupportedZipExtension(path_to_archive)) { #if USE_MINIZIP return std::make_shared(path_to_archive, archive_read_function, archive_size); @@ -36,8 +32,7 @@ std::shared_ptr createArchiveReader( throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - else if (std::any_of( - tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) + else if (hasSupportedTarExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, archive_read_function); @@ -45,7 +40,7 @@ std::shared_ptr createArchiveReader( throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif } - else if (path_to_archive.ends_with(".7z")) + else if (hasSupported7zExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive); diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 9a169587088..53be0a85a10 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -24,10 +25,7 @@ std::shared_ptr createArchiveWriter(const String & path_to_archi std::shared_ptr createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique_ptr archive_write_buffer) { - using namespace std::literals; - static constexpr std::array tar_extensions{ - ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.bz2"sv, ".tar.lzma"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv}; - if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) + if (hasSupportedZipExtension(path_to_archive)) { #if USE_MINIZIP return std::make_shared(path_to_archive, std::move(archive_write_buffer)); @@ -35,8 +33,7 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - else if (std::any_of( - tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) + else if (hasSupportedTarExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, std::move(archive_write_buffer)); diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 4bf7a3ddf86..b9c400d2b98 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -54,10 +55,7 @@ URI::URI(const std::string & uri_) static constexpr auto OSS = "OSS"; static constexpr auto EOS = "EOS"; - if (containsArchive(uri_)) - std::tie(uri_str, archive_pattern) = getPathToArchiveAndArchivePattern(uri_); - else - uri_str = uri_; + std::tie(uri_str, archive_pattern) = getURIAndArchivePattern(uri_); uri = Poco::URI(uri_str); std::unordered_map mapper; @@ -167,32 +165,29 @@ void URI::validateBucket(const String & bucket, const Poco::URI & uri) !uri.empty() ? " (" + uri.toString() + ")" : ""); } -bool URI::containsArchive(const std::string & source) +std::pair> URI::getURIAndArchivePattern(const std::string & source) { size_t pos = source.find("::"); - return (pos != std::string::npos); -} + if (pos == String::npos) + return {source, std::nullopt}; -std::pair URI::getPathToArchiveAndArchivePattern(const std::string & source) -{ - size_t pos = source.find("::"); - assert(pos != std::string::npos); + std::string_view path_to_archive_view = std::string_view{source}.substr(0, pos); + while (path_to_archive_view.ends_with(' ')) + path_to_archive_view.remove_suffix(1); - std::string path_to_archive = source.substr(0, pos); - while ((!path_to_archive.empty()) && path_to_archive.ends_with(' ')) - path_to_archive.pop_back(); + if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + return {source, std::nullopt}; - if (path_to_archive.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + auto archive_uri = path_to_archive_view; - std::string_view path_in_archive_view = std::string_view{source}.substr(pos + 2); - while (path_in_archive_view.front() == ' ') - path_in_archive_view.remove_prefix(1); + std::string_view archive_pattern_view = std::string_view{source}.substr(pos + 2); + while (archive_pattern_view.front() == ' ') + archive_pattern_view.remove_prefix(1); - if (path_in_archive_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); + if (archive_pattern_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Archive pattern is empty"); - return {path_to_archive, std::string{path_in_archive_view}}; + return std::pair{std::string{archive_uri}, std::string{archive_pattern_view}}; } } diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index 363f98c46f5..e4bb0d9eae1 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -42,8 +42,7 @@ struct URI static void validateBucket(const std::string & bucket, const Poco::URI & uri); private: - bool containsArchive(const std::string & source); - std::pair getPathToArchiveAndArchivePattern(const std::string & source); + std::pair> getURIAndArchivePattern(const std::string & source); }; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 98cd5c4dfa9..de56fcf66a0 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -2247,8 +2248,11 @@ void StorageFile::parseFileSource(String source, String & filename, String & pat while (path_to_archive_view.ends_with(' ')) path_to_archive_view.remove_suffix(1); - if (path_to_archive_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + { + filename = std::move(source); + return; + } path_to_archive = path_to_archive_view; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1b6d86f8fa5..12b88ae2b14 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -63,9 +63,12 @@ std::optional TableFunctionFile::tryGetFormatFromFirstArgument() return FormatFactory::instance().tryGetFormatFromFileName(filename); } -StoragePtr TableFunctionFile::getStorage(const String & source, - const String & format_, const ColumnsDescription & columns, - ContextPtr global_context, const std::string & table_name, +StoragePtr TableFunctionFile::getStorage( + const String & source, + const String & format_, + const ColumnsDescription & columns, + ContextPtr global_context, + const std::string & table_name, const std::string & compression_method_) const { // For `file` table function, we are going to use format settings from the diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference new file mode 100644 index 00000000000..243a7c8fd02 --- /dev/null +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference @@ -0,0 +1,12 @@ +::nonexistentfile.csv +1 +nonexistent::nonexistentfile.csv +1 +nonexistent :: nonexistentfile.csv +1 +nonexistent ::nonexistentfile.csv +1 +nonexistent.tar.gz :: nonexistentfile.csv +1 +nonexistent.zip:: nonexistentfile.csv +1 diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh new file mode 100755 index 00000000000..32bf3246c84 --- /dev/null +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function try_to_read_file() +{ + file_to_read=$1 + file_argument=$2 + + echo $file_argument + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument')" 2>&1 | rg -c "Cannot stat file.*$file_to_read" +} + +try_to_read_file "::nonexistentfile.csv" "::nonexistentfile.csv" +try_to_read_file "nonexistent::nonexistentfile.csv" "nonexistent::nonexistentfile.csv" +try_to_read_file "nonexistent :: nonexistentfile.csv" "nonexistent :: nonexistentfile.csv" +try_to_read_file "nonexistent ::nonexistentfile.csv" "nonexistent ::nonexistentfile.csv" +try_to_read_file "nonexistent.tar.gz" "nonexistent.tar.gz :: nonexistentfile.csv" +try_to_read_file "nonexistent.zip" "nonexistent.zip:: nonexistentfile.csv" diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference new file mode 100644 index 00000000000..9dd925a7480 --- /dev/null +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -0,0 +1,2 @@ +::03215_archive.csv test/::03215_archive.csv +test::03215_archive.csv test/test::03215_archive.csv diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql new file mode 100644 index 00000000000..9d01f53c838 --- /dev/null +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -0,0 +1,6 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); +SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } +SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); diff --git a/tests/queries/0_stateless/data_minio/::03215_archive.csv b/tests/queries/0_stateless/data_minio/::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/::03215_archive.csv @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/data_minio/test::03215_archive.csv b/tests/queries/0_stateless/data_minio/test::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test::03215_archive.csv @@ -0,0 +1 @@ +1 From 5381619b2ab465386f11e86242883419e48e5f6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 11:36:25 +0200 Subject: [PATCH 214/496] Remove bad feature. --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 722ae4f8268..a4c873ba3f9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -65,7 +65,6 @@ * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). * The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). -* Allow matching column names in a case-insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). From 4d4fc8fd6f0123613305423d861429f54222d23f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:57:37 +0200 Subject: [PATCH 215/496] Add setting to disable archive path syntax --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/IO/S3/URI.cpp | 8 ++++++-- src/IO/S3/URI.h | 2 +- src/Storages/ObjectStorage/S3/Configuration.cpp | 8 ++++---- src/Storages/StorageFile.cpp | 14 ++++++++++++-- src/Storages/StorageFile.h | 2 +- src/TableFunctions/TableFunctionFile.cpp | 5 +++-- .../03214_parsing_archive_name_file.reference | 4 ++++ .../0_stateless/03214_parsing_archive_name_file.sh | 10 ++++++++-- .../03215_parsing_archive_name_s3.reference | 1 + .../0_stateless/03215_parsing_archive_name_s3.sql | 1 + .../data_minio/test.zip::03215_archive.csv | 1 + 13 files changed, 44 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4fc2034b855..5114a8204cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -936,6 +936,7 @@ class IColumn; M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ + M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..8483a267237 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -124,6 +124,7 @@ static std::initializer_list mapper; diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index e4bb0d9eae1..80e2da96cd4 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -36,7 +36,7 @@ struct URI bool is_virtual_hosted_style; URI() = default; - explicit URI(const std::string & uri_); + explicit URI(const std::string & uri_, bool allow_archive_path_syntax = false); void addRegionToURI(const std::string & region); static void validateBucket(const std::string & bucket, const Poco::URI & uri); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 094ca069e7a..7542f59dcc4 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -142,14 +142,14 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection, ContextPtr context) { - const auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); if (!filename.empty()) - url = S3::URI(std::filesystem::path(collection.get("url")) / filename); + url = S3::URI(std::filesystem::path(collection.get("url")) / filename, settings.allow_archive_path_syntax); else - url = S3::URI(collection.get("url")); + url = S3::URI(collection.get("url"), settings.allow_archive_path_syntax); auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); @@ -330,7 +330,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ } /// This argument is always the first - url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + url = S3::URI(checkAndGetLiteralArgument(args[0], "url"), context->getSettingsRef().allow_archive_path_syntax); if (engine_args_to_idx.contains("format")) { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index de56fcf66a0..efb39f90053 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2208,7 +2208,11 @@ void registerStorageFile(StorageFactory & factory) else if (type == Field::Types::UInt64) source_fd = static_cast(literal->value.get()); else if (type == Field::Types::String) - StorageFile::parseFileSource(literal->value.get(), source_path, storage_args.path_to_archive); + StorageFile::parseFileSource( + literal->value.get(), + source_path, + storage_args.path_to_archive, + factory_args.getLocalContext()->getSettingsRef().allow_archive_path_syntax); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument must be path or file descriptor"); } @@ -2235,8 +2239,14 @@ SchemaCache & StorageFile::getSchemaCache(const ContextPtr & context) return schema_cache; } -void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive) +void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive, bool allow_archive_path_syntax) { + if (!allow_archive_path_syntax) + { + filename = std::move(source); + return; + } + size_t pos = source.find("::"); if (pos == String::npos) { diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 895a8a663b8..bb969c1877c 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -128,7 +128,7 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & context); - static void parseFileSource(String source, String & filename, String & path_to_archive); + static void parseFileSource(String source, String & filename, String & path_to_archive, bool allow_archive_path_syntax); static ArchiveInfo getArchiveInfo( const std::string & path_to_archive, diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 12b88ae2b14..af327cfe54e 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -26,7 +26,7 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr if (context->getApplicationType() != Context::ApplicationType::LOCAL) { ITableFunctionFileLike::parseFirstArguments(arg, context); - StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); + StorageFile::parseFileSource(std::move(filename), filename, path_to_archive, context->getSettingsRef().allow_archive_path_syntax); return; } @@ -42,7 +42,8 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr else if (filename == "stderr") fd = STDERR_FILENO; else - StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); + StorageFile::parseFileSource( + std::move(filename), filename, path_to_archive, context->getSettingsRef().allow_archive_path_syntax); } else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference index 243a7c8fd02..d793d26dfc3 100644 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference @@ -10,3 +10,7 @@ nonexistent.tar.gz :: nonexistentfile.csv 1 nonexistent.zip:: nonexistentfile.csv 1 +nonexistent.tar.gz :: nonexistentfile.csv SETTINGS allow_archive_path_syntax=0 +1 +nonexistent.zip:: nonexistentfile.csv SETTINGS allow_archive_path_syntax=0 +1 diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh index 32bf3246c84..2f77627f6be 100755 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -8,14 +8,20 @@ function try_to_read_file() { file_to_read=$1 file_argument=$2 + settings=$3 - echo $file_argument - $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument')" 2>&1 | rg -c "Cannot stat file.*$file_to_read" + echo $file_argument $settings + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | rg -c "Cannot stat file.*$file_to_read" } +# if archive extension is not detected for part before '::', path is taken as is try_to_read_file "::nonexistentfile.csv" "::nonexistentfile.csv" try_to_read_file "nonexistent::nonexistentfile.csv" "nonexistent::nonexistentfile.csv" try_to_read_file "nonexistent :: nonexistentfile.csv" "nonexistent :: nonexistentfile.csv" try_to_read_file "nonexistent ::nonexistentfile.csv" "nonexistent ::nonexistentfile.csv" +# if archive extension is detected for part before '::', path is split into archive and filename try_to_read_file "nonexistent.tar.gz" "nonexistent.tar.gz :: nonexistentfile.csv" try_to_read_file "nonexistent.zip" "nonexistent.zip:: nonexistentfile.csv" +# disabling archive syntax will always parse path as is +try_to_read_file "nonexistent.tar.gz :: nonexistentfile.csv" "nonexistent.tar.gz :: nonexistentfile.csv" "SETTINGS allow_archive_path_syntax=0" +try_to_read_file "nonexistent.zip:: nonexistentfile.csv" "nonexistent.zip:: nonexistentfile.csv" "SETTINGS allow_archive_path_syntax=0" diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference index 9dd925a7480..b4804c82dc2 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -1,2 +1,3 @@ ::03215_archive.csv test/::03215_archive.csv test::03215_archive.csv test/test::03215_archive.csv +test.zip::03215_archive.csv test/test.zip::03215_archive.csv diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql index 9d01f53c838..3a7ed0b864c 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -4,3 +4,4 @@ SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); +SELECT _file, _path FROM s3(s3_conn, filename='test.zip::03215_archive.csv') ORDER BY (_file, _path) SETTINGS allow_archive_path_syntax=0; diff --git a/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv b/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv @@ -0,0 +1 @@ +1 From b1e80883f0324995d84250d5edf37fd8ab475987 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 11:39:22 +0200 Subject: [PATCH 216/496] `accept_invalid_certificate` in client config: additional testing #65238 --- .../configs/ssl_config_strict.xml | 17 +++++++++ .../test_accept_invalid_certificate/test.py | 35 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml diff --git a/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml b/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml new file mode 100644 index 00000000000..a4383a77ac4 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml @@ -0,0 +1,17 @@ + + + 9440 + + + + + /etc/clickhouse-server/config.d/self-cert.pem + /etc/clickhouse-server/config.d/self-key.pem + /etc/clickhouse-server/config.d/ca-cert.pem + strict + + + diff --git a/tests/integration/test_accept_invalid_certificate/test.py b/tests/integration/test_accept_invalid_certificate/test.py index 87229d75f90..f43e9e6140a 100644 --- a/tests/integration/test_accept_invalid_certificate/test.py +++ b/tests/integration/test_accept_invalid_certificate/test.py @@ -17,6 +17,19 @@ instance = cluster.add_instance( "certs/self-cert.pem", "certs/ca-cert.pem", ], + with_zookeeper=False, +) + + +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/ssl_config_strict.xml", + "certs/self-key.pem", + "certs/self-cert.pem", + "certs/ca-cert.pem", + ], + with_zookeeper=False, ) @@ -90,3 +103,25 @@ def test_connection_accept(): ) == "1\n" ) + + +def test_strict_reject(): + with pytest.raises(Exception) as err: + execute_query_native(node1, "SELECT 1", "") + assert "certificate verify failed" in str(err.value) + + +def test_strict_reject_with_config(): + with pytest.raises(Exception) as err: + execute_query_native(node1, "SELECT 1", config_accept) + assert "alert certificate required" in str(err.value) + + +def test_strict_connection_reject(): + with pytest.raises(Exception) as err: + execute_query_native( + node1, + "SELECT 1", + config_connection_accept.format(ip_address=f"{instance.ip_address}"), + ) + assert "certificate verify failed" in str(err.value) From bdf98cbcc0121ab94dd1db39fc5cf977a7ed42ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Jul 2024 10:06:01 +0000 Subject: [PATCH 217/496] Fix public backports --- tests/ci/cherry_pick.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 0b2aa9a2d35..b660ad2c040 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -505,7 +505,7 @@ class Backport: ReleaseBranch( ( br - if self._repo_name == "ClickHouse/Clickhouse" + if self._repo_name == "ClickHouse/ClickHouse" else f"release/{br}" ), pr, From 9a05a3ed9e7cccada42f49a0cd5c3896010f9edb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:34 +0200 Subject: [PATCH 218/496] Add missing documentation for `groupConcat` after series of reverts and merges https://github.com/ClickHouse/ClickHouse/pull/65384 --- .../reference/groupconcat.md | 90 +++++++++++++++++++ .../aspell-ignore/en/aspell-dict.txt | 2 + 2 files changed, 92 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/groupconcat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md new file mode 100644 index 00000000000..072252de8c9 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -0,0 +1,90 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/groupconcat +sidebar_position: 363 +sidebar_label: groupConcat +title: groupConcat +--- + +Calculates a concatenated string from a group of strings, optionally separated by a delimiter, and optionally limited by a maximum number of elements. + +**Syntax** + +``` sql +groupConcat(expression [, delimiter] [, limit]); +``` + +**Arguments** + +- `expression` — The expression or column name that outputs strings to be concatenated.. +- `delimiter` — A [string](../../../sql-reference/data-types/string.md) that will be used to separate concatenated values. This parameter is optional and defaults to an empty string if not specified. +- `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. + +:::note +If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. +::: + +**Returned value** + +- Returns a [string](../../../sql-reference/data-types/string.md) consisting of the concatenated values of the column or expression. If the group has no elements or only null elements, and the function does not specify a handling for only null values, the result is a nullable string with a null value. + +**Examples** + +Input table: + +``` text +┌─id─┬─name─┐ +│ 1 │ John│ +│ 2 │ Jane│ +│ 3 │ Bob│ +└────┴──────┘ +``` + +1. Basic usage without a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name) FROM Employees; +``` + +Result: + +``` text +JohnJaneBob +``` + +This concatenates all names into one continuous string without any separator. + + +2. Using comma as a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane, Bob +``` + +This output shows the names separated by a comma followed by a space. + + +3. Limiting the number of concatenated elements + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane +``` + +This query limits the output to the first two names, even though there are more names in the table. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b21ae0764c6..d82b70cfdb4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1733,6 +1733,7 @@ groupBitmap groupBitmapAnd groupBitmapOr groupBitmapXor +groupConcat groupUniqArray grouparray grouparrayinsertat @@ -1749,6 +1750,7 @@ groupbitmapor groupbitmapxor groupbitor groupbitxor +groupconcat groupuniqarray grpc grpcio From fb466287dac16801518547f34b42edbc16a57fae Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:13:47 +0200 Subject: [PATCH 219/496] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index be7cfa78492..c29b604d23d 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-distributed-cache +# Tags: long, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d7534c5b40c315cdb3d52101f626dbee7d565cdb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 Jul 2024 12:20:54 +0200 Subject: [PATCH 220/496] Update 03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql --- ...imize_rewrite_aggregate_function_with_if_return_type_bug.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql index b620a6434bf..565a481940a 100644 --- a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql +++ b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + -- For function count, rewrite countState to countStateIf changes the type from AggregateFunction(count, Nullable(UInt64)) to AggregateFunction(count, UInt64) -- We can cast AggregateFunction(count, UInt64) back to AggregateFunction(count, Nullable(UInt64)) with additional _CAST select hex(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; From 6ef628a7c80ebd6ec727365ee69d8141a4f11400 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 Jul 2024 10:33:22 +0000 Subject: [PATCH 221/496] Fixing build. --- src/Processors/QueryPlan/DistinctStep.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index a481454139d..b1c24fc01ce 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static ITransformingStep::Traits getTraits(bool pre_distinct) { const bool preserves_number_of_streams = pre_distinct; @@ -90,7 +95,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil /// final distinct for sorted stream (sorting inside and among chunks) if (input_stream.sort_scope == DataStream::SortScope::Global) { - assert(input_stream.has_single_port); + if (pipeline.getNumStreams() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "DistinctStep with in-order expects single input"); if (distinct_sort_desc.size() < columns.size()) { From 0f8feff4d3806fa6f81d24184ab68bcd6e727551 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:34:30 +0200 Subject: [PATCH 222/496] Add KeeperMap retries --- src/Storages/StorageKeeperMap.cpp | 132 +++++++++++++++++------------- 1 file changed, 73 insertions(+), 59 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index ef157239e26..5534bb7f346 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -35,6 +35,7 @@ #include #include +#include "Common/ZooKeeper/ZooKeeperRetries.h" #include #include #include @@ -120,7 +121,7 @@ public: : SinkToStorage(header), storage(storage_), context(std::move(context_)) { auto primary_key = storage.getPrimaryKey(); - assert(primary_key.size() == 1); + chassert(primary_key.size() == 1); primary_key_pos = getHeader().getPositionByName(primary_key[0]); } @@ -171,76 +172,89 @@ public: template void finalize(bool strict) { - auto zookeeper = storage.getClient(); + const auto & settings = context->getSettingsRef(); - auto keys_limit = storage.keysLimit(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.insert_keeper_max_retries, + settings.insert_keeper_retry_initial_backoff_ms, + settings.insert_keeper_retry_max_backoff_ms}, + context->getProcessListElement()}; - size_t current_keys_num = 0; - size_t new_keys_num = 0; - - // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded - // (e.g if parallel insert queries are being run) - if (keys_limit != 0) + retries_ctl.retryLoop([&]() { - Coordination::Stat data_stat; - zookeeper->get(storage.dataPath(), &data_stat); - current_keys_num = data_stat.numChildren; - } + auto zookeeper = storage.getClient(); + auto keys_limit = storage.keysLimit(); - std::vector key_paths; - key_paths.reserve(new_values.size()); - for (const auto & [key, _] : new_values) - key_paths.push_back(storage.fullPathForKey(key)); + size_t current_keys_num = 0; + size_t new_keys_num = 0; - zkutil::ZooKeeper::MultiExistsResponse results; - - if constexpr (!for_update) - { - if (!strict) - results = zookeeper->exists(key_paths); - } - - Coordination::Requests requests; - requests.reserve(key_paths.size()); - for (size_t i = 0; i < key_paths.size(); ++i) - { - auto key = fs::path(key_paths[i]).filename(); - - if constexpr (for_update) + // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded + // (e.g if parallel insert queries are being run) + if (keys_limit != 0) { - int32_t version = -1; - if (strict) - version = versions.at(key); - - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], version)); + Coordination::Stat data_stat; + zookeeper->get(storage.dataPath(), &data_stat); + current_keys_num = data_stat.numChildren; } - else + + std::vector key_paths; + key_paths.reserve(new_values.size()); + for (const auto & [key, _] : new_values) + key_paths.push_back(storage.fullPathForKey(key)); + + zkutil::ZooKeeper::MultiExistsResponse results; + + if constexpr (!for_update) { - if (!strict && results[i].error == Coordination::Error::ZOK) + if (!strict) + results = zookeeper->exists(key_paths); + } + + Coordination::Requests requests; + requests.reserve(key_paths.size()); + for (size_t i = 0; i < key_paths.size(); ++i) + { + auto key = fs::path(key_paths[i]).filename(); + + if constexpr (for_update) { - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + int32_t version = -1; + if (strict) + version = versions.at(key); + + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], version)); } else { - requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); - ++new_keys_num; + if (!strict && results[i].error == Coordination::Error::ZOK) + { + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); + ++new_keys_num; + } } } - } - if (new_keys_num != 0) - { - auto will_be = current_keys_num + new_keys_num; - if (keys_limit != 0 && will_be > keys_limit) - throw Exception( - ErrorCodes::LIMIT_EXCEEDED, - "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", - new_keys_num, - keys_limit, - will_be); - } + if (new_keys_num != 0) + { + auto will_be = current_keys_num + new_keys_num; + if (keys_limit != 0 && will_be > keys_limit) + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", + new_keys_num, + keys_limit, + will_be); + } - zookeeper->multi(requests, /* check_session_valid */ true); + zookeeper->multi(requests, /* check_session_valid */ true); + }); } }; @@ -529,8 +543,8 @@ Pipe StorageKeeperMap::read( size_t num_keys = keys->size(); size_t num_threads = std::min(num_streams, keys->size()); - assert(num_keys <= std::numeric_limits::max()); - assert(num_threads <= std::numeric_limits::max()); + chassert(num_keys <= std::numeric_limits::max()); + chassert(num_threads <= std::numeric_limits::max()); for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) { @@ -1160,7 +1174,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca bool strict = local_context->getSettingsRef().keeper_map_strict_mode; - assert(commands.size() == 1); + chassert(commands.size() == 1); auto metadata_snapshot = getInMemoryMetadataPtr(); auto storage = getStorageID(); @@ -1236,7 +1250,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca return; } - assert(commands.front().type == MutationCommand::Type::UPDATE); + chassert(commands.front().type == MutationCommand::Type::UPDATE); if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); From 2530c5eb41a759baded5380a4e697c2e884c0abd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:37:35 +0200 Subject: [PATCH 223/496] Fix tests --- tests/queries/0_stateless/02952_archive_parsing.reference | 0 tests/queries/0_stateless/02952_archive_parsing.sql | 1 - tests/queries/0_stateless/03214_parsing_archive_name_file.sh | 2 +- 3 files changed, 1 insertion(+), 2 deletions(-) delete mode 100644 tests/queries/0_stateless/02952_archive_parsing.reference delete mode 100644 tests/queries/0_stateless/02952_archive_parsing.sql diff --git a/tests/queries/0_stateless/02952_archive_parsing.reference b/tests/queries/0_stateless/02952_archive_parsing.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02952_archive_parsing.sql b/tests/queries/0_stateless/02952_archive_parsing.sql deleted file mode 100644 index 49b0223e6ec..00000000000 --- a/tests/queries/0_stateless/02952_archive_parsing.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT * FROM file('::a'); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh index 2f77627f6be..b54cbb10aa6 100755 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -11,7 +11,7 @@ function try_to_read_file() settings=$3 echo $file_argument $settings - $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | rg -c "Cannot stat file.*$file_to_read" + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | grep -c "Cannot stat file.*$file_to_read" } # if archive extension is not detected for part before '::', path is taken as is From 27a15bc5ace68acb0ba62791ca6e3d3f17ae569c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 10:39:35 +0000 Subject: [PATCH 224/496] Remove sh files --- .../03036_dynamic_read_subcolumns_1.sh | 19 ------------------- .../03036_dynamic_read_subcolumns_2.sh | 19 ------------------- .../03036_dynamic_read_subcolumns_3.sh | 19 ------------------- 3 files changed, 57 deletions(-) delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh deleted file mode 100755 index ed548ae74e9..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "Memory" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=Memory" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh deleted file mode 100755 index 95dafcf5832..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh deleted file mode 100755 index a3c2d93e568..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -test -$CH_CLIENT -q "drop table test;" From 72d6467fd2c34a82e1ef8ac73a451240843279a6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 10:43:21 +0000 Subject: [PATCH 225/496] Bump Azure to 1.13 --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index ea3e19a7be0..67272b7ee0a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 +Subproject commit 67272b7ee0adff6b69921b26eb071ba1a353062c From d4537d91875d1ecec832af94fde15073c45a63d7 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:47 +0200 Subject: [PATCH 226/496] Add `groupConcat` to fuzzer https://github.com/ClickHouse/ClickHouse/pull/65384 --- tests/fuzz/dictionaries/functions.dict | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index ec7f8017fb2..6f2a88c22fa 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -1588,6 +1588,7 @@ "groupBitmapXorResample" "groupBitmapXorSimpleState" "groupBitmapXorState" +"groupConcat" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" From d4d3d590e38436da44b13dbf11a92cc6d00863e7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 20:06:55 +0000 Subject: [PATCH 227/496] Remove obsolete --multiquery parameter from tests --- .../integration/test_distributed_format/test.py | 8 ++++---- ...80_client_break_at_exception_in_batch_mode.sh | 2 +- .../00463_long_sessions_in_http_interface.sh | 10 +++++----- .../0_stateless/00474_readonly_settings.sh | 4 ++-- .../0_stateless/00612_pk_in_tuple_perf.sh | 4 ++-- .../0_stateless/00630_arbitrary_csv_delimiter.sh | 6 +++--- .../00650_csv_with_specified_quote_rule.sh | 4 ++-- ...00651_default_database_on_client_reconnect.sh | 2 +- .../00652_mutations_default_database.sh | 2 +- ...cated_mutations_default_database_zookeeper.sh | 2 +- .../00699_materialized_view_mutations.sh | 8 ++++---- .../00704_drop_truncate_memory_table.sh | 6 +++--- .../0_stateless/00705_drop_create_merge_tree.sh | 4 ++-- .../0_stateless/00763_lock_buffer_long.sh | 4 ++-- ...3_long_lock_buffer_alter_destination_table.sh | 4 ++-- .../00825_protobuf_format_array_3dim.sh | 2 +- .../00825_protobuf_format_array_of_arrays.sh | 2 +- .../00825_protobuf_format_enum_mapping.sh | 2 +- .../0_stateless/00825_protobuf_format_map.sh | 2 +- .../00825_protobuf_format_nested_in_nested.sh | 2 +- .../00825_protobuf_format_nested_optional.sh | 2 +- .../00825_protobuf_format_no_length_delimiter.sh | 6 +++--- .../0_stateless/00825_protobuf_format_persons.sh | 4 ++-- ...5_protobuf_format_skipped_column_in_nested.sh | 2 +- .../00825_protobuf_format_splitted_nested.sh | 2 +- .../0_stateless/00825_protobuf_format_squares.sh | 2 +- .../00825_protobuf_format_table_default.sh | 2 +- .../0_stateless/00900_long_parquet_load.sh | 2 +- .../0_stateless/00900_orc_arrow_parquet_maps.sh | 2 +- .../00937_format_schema_rows_template.sh | 4 ++-- .../0_stateless/00956_sensitive_data_masking.sh | 12 ++++++------ .../01019_alter_materialized_view_atomic.sh | 2 +- .../01019_alter_materialized_view_consistent.sh | 2 +- .../0_stateless/01035_lc_empty_part_bug.sh | 4 ++-- ...ystem_reload_dictionary_reloads_completely.sh | 2 +- .../01052_window_view_proc_tumble_to_now.sh | 2 +- .../01053_window_view_proc_hop_to_now.sh | 2 +- .../01054_window_view_proc_tumble_to.sh | 2 +- .../0_stateless/01055_window_view_proc_hop_to.sh | 2 +- ...057_window_view_event_tumble_to_strict_asc.sh | 2 +- .../01058_window_view_event_hop_to_strict_asc.sh | 2 +- .../01060_window_view_event_tumble_to_asc.sh | 2 +- .../01061_window_view_event_hop_to_asc.sh | 2 +- .../01063_window_view_event_tumble_to_bounded.sh | 2 +- .../01064_window_view_event_hop_to_bounded.sh | 2 +- ...w_view_event_tumble_to_strict_asc_lateness.sh | 2 +- ...7_window_view_event_tumble_to_asc_lateness.sh | 2 +- ...ndow_view_event_tumble_to_bounded_lateness.sh | 2 +- .../01071_window_view_event_tumble_asc_join.sh | 2 +- ...01072_window_view_multiple_columns_groupby.sh | 2 +- ...3_window_view_event_tumble_to_asc_populate.sh | 4 ++-- ...window_view_event_tumble_asc_join_populate.sh | 2 +- ...75_window_view_proc_tumble_to_now_populate.sh | 2 +- .../01076_window_view_alter_query_to.sh | 4 ++-- ...7_window_view_alter_query_to_modify_source.sh | 4 ++-- ...1079_window_view_inner_table_memory_tumble.sh | 2 +- .../01080_window_view_inner_table_memory_hop.sh | 2 +- .../01081_window_view_target_table_engine.sh | 2 +- .../0_stateless/01083_window_view_select.sh | 2 +- .../01084_window_view_with_table_identifier.sh | 2 +- .../0_stateless/01086_window_view_cleanup.sh | 2 +- .../0_stateless/01087_window_view_alter_query.sh | 4 ++-- .../01088_window_view_default_column.sh | 2 +- .../0_stateless/01133_begin_commit_race.sh | 6 +++--- .../01169_alter_partition_isolation_stress.sh | 4 ++-- ...01169_old_alter_partition_isolation_stress.sh | 10 +++++----- .../01171_mv_select_insert_isolation_long.sh | 14 +++++++------- .../0_stateless/01174_select_insert_isolation.sh | 8 ++++---- .../0_stateless/01198_client_quota_key.sh | 2 +- .../0_stateless/01285_engine_join_donmikel.sh | 4 ++-- .../0_stateless/01293_optimize_final_force.sh | 2 +- .../queries/0_stateless/01304_direct_io_long.sh | 4 ++-- ...orage_file_tsv_csv_with_names_write_prefix.sh | 4 ++-- .../0_stateless/01443_merge_truncate_long.sh | 2 +- .../01527_clickhouse_local_optimize.sh | 2 +- .../01543_avro_deserialization_with_lc.sh | 2 +- .../0_stateless/01544_file_engine_settings.sh | 4 ++-- .../0_stateless/01600_detach_permanently.sh | 2 +- .../01600_parts_states_metrics_long.sh | 2 +- tests/queries/0_stateless/01606_git_import.sh | 4 ++-- .../0_stateless/01607_arrays_as_nested_csv.sh | 4 ++-- .../0_stateless/01632_tinylog_read_write.sh | 4 ++-- .../01658_read_file_to_stringcolumn.sh | 4 ++-- .../01666_merge_tree_max_query_limit.sh | 2 +- .../0_stateless/01747_system_session_log_long.sh | 6 +++--- .../01801_nullable_low_cardinality_tsv.sh | 2 +- .../01834_alias_columns_laziness_filimonov.sh | 2 +- .../01923_network_receive_time_metric_insert.sh | 4 ++-- .../01939_network_receive_bytes_metrics.sh | 4 ++-- .../01946_test_wrong_host_name_access.sh | 2 +- ..._with_escape_sequence_at_the_end_of_buffer.sh | 2 +- tests/queries/0_stateless/02009_from_infile.sh | 2 +- .../0_stateless/02024_compression_in_query.sh | 6 +++--- .../02048_parallel_reading_from_infile.sh | 6 +++--- ...02104_clickhouse_local_columns_description.sh | 2 +- ...insert_deduplication_token_multiple_blocks.sh | 10 +++++----- ...eduplication_token_multiple_blocks_replica.sh | 10 +++++----- .../queries/0_stateless/02125_many_mutations.sh | 6 +++--- .../0_stateless/02125_many_mutations_2.sh | 6 +++--- .../queries/0_stateless/02135_local_create_db.sh | 2 +- .../0_stateless/02151_client_option_echo.sh | 4 ++-- .../0_stateless/02151_hash_table_sizes_stats.sh | 2 +- .../02151_hash_table_sizes_stats_distributed.sh | 2 +- .../02158_explain_ast_alter_commands.sh | 2 +- .../02206_clickhouse_local_use_database.sh | 2 +- .../02226_filesystem_cache_profile_events.sh | 14 +++++++------- .../02227_test_create_empty_sqlite_db.sh | 4 ++-- .../0_stateless/02235_remote_fs_cache_stress.sh | 6 +++--- .../02240_protobuflist_format_persons.sh | 4 ++-- .../02246_clickhouse_local_drop_database.sh | 4 ++-- .../0_stateless/02286_drop_filesystem_cache.sh | 2 +- .../02337_drop_filesystem_cache_access.sh | 8 ++++---- .../0_stateless/02364_window_view_segfault.sh | 2 +- .../0_stateless/02373_datetime64_monotonicity.sh | 2 +- .../0_stateless/02416_rename_database_rbac.sh | 4 ++-- .../02435_rollback_cancelled_queries.sh | 2 +- ...60_projections_and_aggregate_null_if_empty.sh | 2 +- .../02494_query_cache_user_isolation.sh | 8 ++++---- ...503_cache_on_write_with_small_segment_size.sh | 4 ++-- .../queries/0_stateless/02521_merge_over_gap.sh | 2 +- .../02530_dictionaries_update_field.sh | 2 +- .../0_stateless/02702_allow_skip_errors_enum.sh | 2 +- .../0_stateless/02704_keeper_map_zk_nodes.sh | 2 +- ...ckhouse_local_implicit_file_table_function.sh | 4 ++-- .../02712_bool_better_exception_message.sh | 8 ++++---- .../0_stateless/02722_database_filesystem.sh | 8 ++++---- tests/queries/0_stateless/02724_database_s3.sh | 14 +++++++------- tests/queries/0_stateless/02725_database_hdfs.sh | 10 +++++----- .../0_stateless/02725_local_query_parameters.sh | 2 +- .../02751_multiquery_with_argument.reference | 2 -- .../02751_multiquery_with_argument.sh | 16 ++++++---------- .../02815_no_throw_in_simple_queries.sh | 2 +- .../02843_insertion_table_schema_infer.sh | 2 +- .../02864_restore_table_with_broken_part.sh | 2 +- ...ultiple_batches_array_inconsistent_offsets.sh | 2 +- .../0_stateless/02875_merge_engine_set_index.sh | 2 +- .../02877_optimize_read_in_order_from_view.sh | 2 +- ...02884_create_view_with_sql_security_option.sh | 16 ++++++++-------- ...885_async_insert_access_check_for_defaults.sh | 4 ++-- ...900_clickhouse_local_drop_current_database.sh | 2 +- .../02956_clickhouse_local_system_parts.sh | 2 +- .../02973_backup_of_in_memory_compressed.sh | 10 +++++----- .../02973_parse_crlf_with_tsv_files.sh | 2 +- .../0_stateless/02995_forget_partition.sh | 4 ++-- tests/queries/0_stateless/02995_index_1.sh | 6 +++--- tests/queries/0_stateless/02995_index_10.sh | 6 +++--- tests/queries/0_stateless/02995_index_2.sh | 6 +++--- tests/queries/0_stateless/02995_index_3.sh | 6 +++--- tests/queries/0_stateless/02995_index_4.sh | 6 +++--- tests/queries/0_stateless/02995_index_5.sh | 6 +++--- tests/queries/0_stateless/02995_index_6.sh | 6 +++--- tests/queries/0_stateless/02995_index_7.sh | 6 +++--- tests/queries/0_stateless/02995_index_8.sh | 6 +++--- tests/queries/0_stateless/02995_index_9.sh | 6 +++--- .../0_stateless/02998_native_parquet_reader.sh | 2 +- .../03001_backup_matview_after_modify_query.sh | 2 +- .../03001_matview_columns_after_modify_query.sh | 2 +- .../03006_correct_revoke_for_partial_rights.sh | 2 +- .../03147_system_columns_access_checks.sh | 4 ++-- .../0_stateless/03201_local_named_collections.sh | 6 +++--- .../0_stateless/03212_thousand_exceptions.sh | 2 +- 161 files changed, 316 insertions(+), 322 deletions(-) diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 91afb8f7b34..5611f465e8b 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -55,7 +55,7 @@ def test_single_file(started_cluster, cluster): path = get_dist_path(cluster, "distr_1", 1) query = f"select * from file('{path}/1.bin', 'Distributed')" out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" @@ -65,7 +65,7 @@ def test_single_file(started_cluster, cluster): select * from t; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" @@ -106,7 +106,7 @@ def test_two_files(started_cluster, cluster): select * from t order by x; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "0\t_\n1\ta\n2\tbb\n3\tccc\n" @@ -141,7 +141,7 @@ def test_single_file_old(started_cluster, cluster): select * from t; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" diff --git a/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh b/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh index 62f891db33c..0aab52d15c2 100755 --- a/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh +++ b/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query="SELECT 1; SELECT xyz; SELECT 2;" 2> /dev/null || true; +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT xyz; SELECT 2;" 2> /dev/null || true; diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh index d41d6409315..6ee1649c9ed 100755 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh @@ -25,7 +25,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_4&se ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_5&session_timeout=60" --data-binary "SELECT 1" echo "Sessions are local per user:" -${CLICKHOUSE_CLIENT} --multiquery --query "DROP USER IF EXISTS test_00463; CREATE USER test_00463; GRANT ALL ON *.* TO test_00463;" +${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS test_00463; CREATE USER test_00463; GRANT ALL ON *.* TO test_00463;" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6&session_timeout=600" --data-binary "CREATE TEMPORARY TABLE t (s String)" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "INSERT INTO t VALUES ('Hello')" @@ -37,7 +37,7 @@ ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${C ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" -${CLICKHOUSE_CLIENT} --multiquery --query "DROP USER test_00463"; +${CLICKHOUSE_CLIENT} --query "DROP USER test_00463"; echo "And cannot be accessed for a non-existent user:" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" | grep -c -F 'Exception' @@ -59,7 +59,7 @@ done echo "A session successfully expire after a timeout and the session's temporary table shadows the permanent table:" # An infinite loop is required to make the test reliable. We will check that the timeout corresponds to the observed time at least once -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (s String) ENGINE = Memory; INSERT INTO t VALUES ('World');" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (s String) ENGINE = Memory; INSERT INTO t VALUES ('World');" while true do ( @@ -70,7 +70,7 @@ do ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_8" --data-binary "SELECT * FROM t" ) | tr -d '\n' | grep -F 'HelloWorld' && break || sleep 1 done -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE t" +${CLICKHOUSE_CLIENT} --query "DROP TABLE t" echo "A session cannot be used by concurrent connections:" @@ -83,5 +83,5 @@ do done ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT 1" | grep -c -F 'SESSION_IS_LOCKED' -${CLICKHOUSE_CLIENT} --multiquery --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; +${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; wait diff --git a/tests/queries/0_stateless/00474_readonly_settings.sh b/tests/queries/0_stateless/00474_readonly_settings.sh index 3a857d81a74..ed3558c6d7a 100755 --- a/tests/queries/0_stateless/00474_readonly_settings.sh +++ b/tests/queries/0_stateless/00474_readonly_settings.sh @@ -8,8 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT --readonly=1 --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT --readonly=1 --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value diff --git a/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh b/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh index c8297635c43..7b2973669de 100755 --- a/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh +++ b/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -$CLICKHOUSE_CLIENT --multiquery </dev/null ||: +${CLICKHOUSE_CLIENT} --ignore-error --query "DROP TABLE IF EXISTS tab_00651; CREATE TABLE tab_00651 (val UInt64) engine = Memory; SHOW CREATE TABLE tab_00651 format abcd; DESC tab_00651; DROP TABLE tab_00651;" 2>/dev/null ||: diff --git a/tests/queries/0_stateless/00652_mutations_default_database.sh b/tests/queries/0_stateless/00652_mutations_default_database.sh index eed45540f9b..577943bc3fd 100755 --- a/tests/queries/0_stateless/00652_mutations_default_database.sh +++ b/tests/queries/0_stateless/00652_mutations_default_database.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --mutations_sync=1 << EOF +${CLICKHOUSE_CLIENT} --mutations_sync=1 << EOF DROP TABLE IF EXISTS mutations; DROP TABLE IF EXISTS for_subquery; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh index 0ac5a2f748a..d4f6d3b290c 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 --multiquery << EOF +${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 << EOF DROP TABLE IF EXISTS mutations_r1; DROP TABLE IF EXISTS for_subquery; diff --git a/tests/queries/0_stateless/00699_materialized_view_mutations.sh b/tests/queries/0_stateless/00699_materialized_view_mutations.sh index a0f7db536dc..07ca9bc0f67 100755 --- a/tests/queries/0_stateless/00699_materialized_view_mutations.sh +++ b/tests/queries/0_stateless/00699_materialized_view_mutations.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS view_00699; DROP TABLE IF EXISTS null_00699; @@ -20,14 +20,14 @@ SELECT count(), min(x), max(x) FROM view_00699; ALTER TABLE null_00699 DELETE WHERE x % 2 = 0;" --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; ALTER TABLE view_00699 DELETE WHERE x % 2 = 0; " --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; @@ -35,7 +35,7 @@ ALTER TABLE null_00699 DELETE WHERE x % 2 = 1; ALTER TABLE view_00699 DELETE WHERE x % 2 = 1; " --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; diff --git a/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh b/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh index e1540d1a25e..e40da11b893 100755 --- a/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh +++ b/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS memory; CREATE TABLE memory (x UInt64) ENGINE = Memory; @@ -21,13 +21,13 @@ INSERT INTO memory SELECT * FROM numbers(1000);" # But if the table will be dropped before query - just pass. # It's Ok, because otherwise the test will depend on the race condition in the test itself. -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SET max_threads = 1; SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|^0$|Exception' & sleep 0.05; -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" TRUNCATE TABLE memory; DROP TABLE memory; " diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index ea8b9d02e49..fd002668696 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & -yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & +yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT & +yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT & wait ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table" diff --git a/tests/queries/0_stateless/00763_lock_buffer_long.sh b/tests/queries/0_stateless/00763_lock_buffer_long.sh index 444a66767aa..92f917aa287 100755 --- a/tests/queries/0_stateless/00763_lock_buffer_long.sh +++ b/tests/queries/0_stateless/00763_lock_buffer_long.sh @@ -16,12 +16,12 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_2 (s String) ENGINE = Bu function thread1() { - seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --fsync-metadata 0 --multiquery --ignore-error ||: + seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --fsync-metadata 0 --ignore-error ||: } function thread2() { - seq 1 500 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' + seq 1 500 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' } thread1 & diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 7e2384cfc52..79df667d45f 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), function thread1() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --ignore-error ||: } function thread2() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' } thread1 & diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh index 3cd842a10ba..468ced802cd 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh @@ -9,7 +9,7 @@ SCHEMADIR=$CURDIR/format_schemas set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery < /dev/null < /dev/null < $row_format_file -$CLICKHOUSE_CLIENT --multiline --multiquery --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ +$CLICKHOUSE_CLIENT --multiline --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ format_template_row = '$row_format_file', \ format_template_row_format = 'Question: \${question:Quoted}, Answer: \${answer:Quoted}, Number of Likes: \${likes:Raw}, Date: \${date:Raw}', \ format_template_rows_between_delimiter = ';\n'; --{clientError 474}" @@ -38,7 +38,7 @@ format_template_rows_between_delimiter = ';\n'"; # Test that if both format_template_result_format setting and format_template_resultset are provided, error is thrown resultset_output_file="$CURDIR"/"$CLICKHOUSE_TEST_UNIQUE_NAME"_template_output_format_resultset.tmp echo -ne '===== Resultset ===== \n \${data} \n ===============' > $resultset_output_file -$CLICKHOUSE_CLIENT --multiline --multiquery --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ +$CLICKHOUSE_CLIENT --multiline --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ format_template_resultset = '$resultset_output_file', \ format_template_resultset_format = '===== Resultset ===== \n \${data} \n ===============', \ format_template_row_format = 'Question: \${question:Quoted}, Answer: \${answer:Quoted}, Number of Likes: \${likes:Raw}, Date: \${date:Raw}', \ diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index 926557e4ba6..bd65b937648 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -17,7 +17,7 @@ echo 1 # normal execution $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM numbers(1) FORMAT Null" \ - --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 + --log_queries=1 --ignore-error >"$tmp_file" 2>&1 grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 1a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 1b' @@ -38,7 +38,7 @@ echo 3 # failure at before query start $CLICKHOUSE_CLIENT \ --query="SELECT 1 FROM system.numbers WHERE credit_card_number='find_me_TOPSECRET=TOPSECRET' FORMAT Null" \ - --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > "$tmp_file" + --log_queries=1 --ignore-error |& grep -v '^(query: ' > "$tmp_file" grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 3a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 3b' @@ -56,7 +56,7 @@ echo 4 # failure at the end of query $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET', intDiv( 100, number - 10) FROM numbers(11) FORMAT Null" \ - --log_queries=1 --ignore-error --max_block_size=2 --multiquery |& grep -v '^(query: ' > "$tmp_file" + --log_queries=1 --ignore-error --max_block_size=2 |& grep -v '^(query: ' > "$tmp_file" grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 4a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 4b' @@ -67,7 +67,7 @@ rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ --function_sleep_max_microseconds_per_block 60000000 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ - --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & + --log_queries=1 --ignore-error |& grep -v '^(query: ' > $tmp_file2" & rm -f "$tmp_file" >/dev/null 2>&1 # check that executing query doesn't expose secrets in processlist @@ -133,7 +133,7 @@ insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' insert into sensitive select number as id, toDate('2019-01-01') as date, 'find_me_TOPSECRET=TOPSECRET' as value1, rand() as valuer from numbers(10); insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' as value1, rand() as valuer from numbers(10000); select * from sensitive WHERE value1 = 'find_me_TOPSECRET=TOPSECRET' FORMAT Null; -drop table sensitive;" --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 +drop table sensitive;" --log_queries=1 --ignore-error >"$tmp_file" 2>&1 grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 8a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 8b' @@ -144,7 +144,7 @@ echo 9 $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ --query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_date >= yesterday() and message like '%find_me%'; - select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery + select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error echo 'finish' rm -f "$tmp_file" >/dev/null 2>&1 diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index 4bd21fcee02..eb12a76eb62 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery <&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION @@ -24,7 +24,7 @@ function begin_commit_readonly() function begin_rollback_readonly() { while true; do - $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=wait_unknown --multiquery --query " + $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=wait_unknown --query " BEGIN TRANSACTION; SET TRANSACTION SNAPSHOT 42; ROLLBACK;" @@ -34,7 +34,7 @@ function begin_rollback_readonly() function begin_insert_commit() { while true; do - $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=async --multiquery --query " + $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=async --query " BEGIN TRANSACTION; INSERT INTO mt VALUES ($RANDOM); COMMIT;" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index d4884cbf457..8873fd88f0e 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -22,7 +22,7 @@ function thread_insert() set -eu val=1 while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); @@ -210,7 +210,7 @@ function thread_select() set -eu while true; do output=$( - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; -- no duplicates SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh index 0d2016952d4..404042ab64e 100755 --- a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -19,7 +19,7 @@ function thread_insert() set -e val=1 while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); @@ -40,7 +40,7 @@ function thread_partition_src_to_dst() sum=0 for i in {1..20}; do out=$( - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($i, 3) */ ($i, 3); INSERT INTO dst SELECT * FROM src; @@ -49,7 +49,7 @@ function thread_partition_src_to_dst() SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; COMMIT;" 2>&1) ||: - echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --multiquery --query " + echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --query " begin transaction; set transaction snapshot 3; select $i, 'src', type, n, _part from src order by type, n; @@ -68,7 +68,7 @@ function thread_partition_dst_to_src() if (( i % 2 )); then action="COMMIT" fi - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " SYSTEM STOP MERGES dst; ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick SYSTEM SYNC TRANSACTION LOG; @@ -87,7 +87,7 @@ function thread_select() { set -e while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; -- no duplicates SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 3b9bb50517d..2fb58e4cc57 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -65,7 +65,7 @@ function insert_commit_action() local tag=$1; shift # some transactions will fail due to constraint - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($i, $tag) */ ($i, $tag); SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $tag))) /* ($i, $tag) */ FORMAT Null; @@ -83,7 +83,7 @@ function insert_rollback_action() local i=$1; shift local tag=$1; shift - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* (42, $tag) */ (42, $tag); SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$tag) != 1) FORMAT Null; @@ -112,7 +112,7 @@ function optimize_action() action="ROLLBACK" fi - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; $optimize_query; $action; @@ -126,7 +126,7 @@ function select_action() { set -e - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; @@ -140,7 +140,7 @@ function select_insert_action() { set -e - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null; INSERT INTO tmp SELECT 1, n*m FROM src; @@ -199,7 +199,7 @@ wait $PID_8 || echo "second select_insert_action has failed with status $?" 2>&1 wait_for_queries_to_finish $WAIT_FINISH -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; @@ -209,7 +209,7 @@ $CLICKHOUSE_CLIENT --multiquery --query " COMMIT; " -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 6321f6ff01b..235d98fb5de 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree OR function thread_insert_commit() { for i in {1..50}; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO mt VALUES /* ($i, $1) */ ($i, $1); INSERT INTO mt VALUES /* (-$i, $1) */ (-$i, $1); @@ -27,7 +27,7 @@ function thread_insert_commit() function thread_insert_rollback() { for _ in {1..50}; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO mt VALUES /* (42, $1) */ (42, $1); ROLLBACK;"; @@ -38,7 +38,7 @@ function thread_select() { while true; do # The first and the last queries must get the same result - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SET throw_on_unsupported_query_inside_transaction=0; CREATE TEMPORARY TABLE tmp AS SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt FORMAT Null; @@ -58,7 +58,7 @@ kill -TERM $PID_4 wait wait_for_queries_to_finish 40 -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; diff --git a/tests/queries/0_stateless/01198_client_quota_key.sh b/tests/queries/0_stateless/01198_client_quota_key.sh index 3f5f5df5071..d08aa2e364f 100755 --- a/tests/queries/0_stateless/01198_client_quota_key.sh +++ b/tests/queries/0_stateless/01198_client_quota_key.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'" +$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'" diff --git a/tests/queries/0_stateless/01285_engine_join_donmikel.sh b/tests/queries/0_stateless/01285_engine_join_donmikel.sh index 7522ed9924b..ce273ab8e0c 100755 --- a/tests/queries/0_stateless/01285_engine_join_donmikel.sh +++ b/tests/queries/0_stateless/01285_engine_join_donmikel.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS NmSubj; DROP TABLE IF EXISTS events; @@ -60,7 +60,7 @@ FROM events as e INNER JOIN NmSubj as ns ON ns.NmId = toUInt32(e.Param1) WHERE e.EventDate = today() - 7 AND e.EventId = 'GCO' AND ns.SubjectId = 2073" -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE NmSubj; DROP TABLE events; " diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index e838af8af9b..9c135d272e4 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -11,7 +11,7 @@ TIMELIMIT=31 while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ]; do it=$((it+1)) - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt8, k UInt8 DEFAULT 0) ENGINE = SummingMergeTree ORDER BY k; diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index 2e27c2f7728..1241f299d94 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS bug; CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi', merge_max_block_size = 8192; @@ -18,5 +18,5 @@ cat "$LOG" | grep Loaded rm "$LOG" -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE bug;" diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh index af8d3f4e69b..a634f689dca 100755 --- a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo 'zero rows' for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format - ${CLICKHOUSE_LOCAL} --multiquery --query=" + ${CLICKHOUSE_LOCAL} --query=" CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1) WHERE number < 0; SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; @@ -22,7 +22,7 @@ echo 'multi clickhouse-local one file' for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format for _ in {1..2}; do - ${CLICKHOUSE_LOCAL} --multiquery --query=" + ${CLICKHOUSE_LOCAL} --query=" CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1); SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.sh b/tests/queries/0_stateless/01443_merge_truncate_long.sh index 65b9bcd366e..51654b2e4e1 100755 --- a/tests/queries/0_stateless/01443_merge_truncate_long.sh +++ b/tests/queries/0_stateless/01443_merge_truncate_long.sh @@ -34,7 +34,7 @@ do SELECT count() FROM t HAVING count() > 0; SELECT ${i}; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} wait diff --git a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh index d61d34244d9..c1d5c357308 100755 --- a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh +++ b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh @@ -10,6 +10,6 @@ rm -rf "${WORKING_FOLDER_01527}" mkdir -p "${WORKING_FOLDER_01527}" # OPTIMIZE was crashing due to lack of temporary volume in local -${CLICKHOUSE_LOCAL} --multiquery --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}" +${CLICKHOUSE_LOCAL} --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}" rm -rf "${WORKING_FOLDER_01527}" diff --git a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh index a5697a62dc2..bc9efaedd5d 100755 --- a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh +++ b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " SET allow_suspicious_low_cardinality_types=1; CREATE TABLE IF NOT EXISTS test_01543 (value LowCardinality(String), value2 LowCardinality(UInt64)) ENGINE=Memory(); " diff --git a/tests/queries/0_stateless/01544_file_engine_settings.sh b/tests/queries/0_stateless/01544_file_engine_settings.sh index b31754f9531..eb0a8a964d0 100755 --- a/tests/queries/0_stateless/01544_file_engine_settings.sh +++ b/tests/queries/0_stateless/01544_file_engine_settings.sh @@ -10,7 +10,7 @@ rm -f -- "$the_file" # We are going to check that format settings work for File engine, # by creating a table with a non-default delimiter, and reading from it. -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; insert into t select 1 a, 1 b; " @@ -18,7 +18,7 @@ ${CLICKHOUSE_LOCAL} --multiquery --query " # See what's in the file cat "$the_file" -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; select * from t; " diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index 6721dbf3015..679e9a749ee 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -18,7 +18,7 @@ mkdir -p "${WORKING_FOLDER_01600}" clickhouse_local() { local query="$1" shift - ${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --multiquery --query "$query" "$@" --path="${WORKING_FOLDER_01600}" + ${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --query "$query" "$@" --path="${WORKING_FOLDER_01600}" } test_detach_attach_sequence() { diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 7215f270a4c..47b5a4dea13 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -24,7 +24,7 @@ verify() if [[ $i -eq 5000 ]] then - $CLICKHOUSE_CLIENT --multiquery " + $CLICKHOUSE_CLIENT " SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; SELECT sum(active), sum(NOT active) FROM system.parts; SELECT sum(active), sum(NOT active) FROM system.projection_parts; diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index 48558d79f93..6986d6b14cf 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -19,7 +19,7 @@ done ${CLICKHOUSE_GIT_IMPORT} 2>&1 | wc -l -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " DROP TABLE IF EXISTS commits; DROP TABLE IF EXISTS file_changes; @@ -122,7 +122,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM commits" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM file_changes" ${CLICKHOUSE_CLIENT} --query "SELECT count(), round(avg(indent), 1) FROM line_changes" -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " DROP TABLE commits; DROP TABLE file_changes; DROP TABLE line_changes; diff --git a/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh b/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh index 946be7fb4af..2a1182c14c1 100755 --- a/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh +++ b/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS test; CREATE TABLE test (a Array(String)) ENGINE = Memory; " @@ -22,7 +22,7 @@ ${CLICKHOUSE_CLIENT} --input_format_csv_arrays_as_nested_csv 1 --query "INSERT I """Hello"", ""world"", ""42"""" TV""" END -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM test; DROP TABLE IF EXISTS test; " diff --git a/tests/queries/0_stateless/01632_tinylog_read_write.sh b/tests/queries/0_stateless/01632_tinylog_read_write.sh index 10625ec5d27..68d28b080e9 100755 --- a/tests/queries/0_stateless/01632_tinylog_read_write.sh +++ b/tests/queries/0_stateless/01632_tinylog_read_write.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test; CREATE TABLE IF NOT EXISTS test (x UInt64, s Array(Nullable(String))) ENGINE = TinyLog;" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test; CREATE TABLE IF NOT EXISTS test (x UInt64, s Array(Nullable(String))) ENGINE = TinyLog;" function thread_select { local TIMELIMIT=$((SECONDS+$1)) @@ -47,4 +47,4 @@ thread_insert $TIMEOUT & wait echo "Done" -$CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test;" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test;" diff --git a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh index 685fe69642a..ceb6aa060ea 100755 --- a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh +++ b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh @@ -24,7 +24,7 @@ ${CLICKHOUSE_CLIENT} --query "select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "select file('c.txt'), * from data";echo ":"$? -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " create table filenames(name String) engine=MergeTree() order by tuple(); insert into filenames values ('a.txt'), ('b.txt'), ('c.txt'); select file(name) from filenames format TSV; @@ -56,7 +56,7 @@ echo $c_count # Valid cases: # The default dir is the CWD path in LOCAL mode -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " drop table if exists data; create table data (A String, B String) engine=MergeTree() order by A; select file('a.txt'), file('b.txt'); diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index e04c9515009..ec318db98bf 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -8,7 +8,7 @@ function wait_for_query_to_start() { while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT sum(read_rows) FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done } -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " drop table if exists simple; create table simple (i int, j int) engine = MergeTree order by i diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index 022bf488886..07055f96782 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -82,7 +82,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" + ! ${CLICKHOUSE_CLIENT} --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" } function createUser() @@ -303,7 +303,7 @@ function runEndpointTests() if [[ -n "${setup_queries}" ]] then # echo "Executing setup queries: ${setup_queries}" - echo "${setup_queries}" | executeQuery --multiquery + echo "${setup_queries}" | executeQuery fi testTCP "${auth_type}" "${username}" "${password}" @@ -357,7 +357,7 @@ testAsUserIdentifiedBy "plaintext_password" testAsUserIdentifiedBy "sha256_password" testAsUserIdentifiedBy "double_sha1_password" -executeQuery --multiquery <= 1000000 ? 1 : time FROM system.query_log WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" diff --git a/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh b/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh index 03babad40f3..b2335a0365b 100755 --- a/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh +++ b/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" seq 1 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" -${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; WITH ProfileEvents['NetworkReceiveBytes'] AS bytes SELECT bytes >= 8000 AND bytes < 9000 ? 1 : bytes FROM system.query_log WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" diff --git a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh index a00f307673e..ed2828c3f54 100755 --- a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh +++ b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " DROP USER IF EXISTS dns_fail_1, dns_fail_2; CREATE USER dns_fail_1 HOST NAME 'non.existing.host.name', '${MYHOSTNAME}'; CREATE USER dns_fail_2 HOST NAME '${MYHOSTNAME}', 'non.existing.host.name';" diff --git a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh index 0aedef028a2..b3748581f4f 100755 --- a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh +++ b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) mkdir -p ${USER_FILES_PATH}/ cp $CUR_DIR/data_zstd/test_01946.zstd ${USER_FILES_PATH}/ -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " set min_chunk_bytes_for_parallel_parsing=10485760; set max_read_buffer_size = 65536; set input_format_parallel_parsing = 0; diff --git a/tests/queries/0_stateless/02009_from_infile.sh b/tests/queries/0_stateless/02009_from_infile.sh index 6a31aa4ac55..578ac14f558 100755 --- a/tests/queries/0_stateless/02009_from_infile.sh +++ b/tests/queries/0_stateless/02009_from_infile.sh @@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_infile;" # if it not fails, select will print information -${CLICKHOUSE_LOCAL} --multiquery --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;" +${CLICKHOUSE_LOCAL} --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=DROP+TABLE" -d 'IF EXISTS test_infile_url' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=CREATE" -d 'TABLE test_infile_url (x String) ENGINE = Memory' diff --git a/tests/queries/0_stateless/02024_compression_in_query.sh b/tests/queries/0_stateless/02024_compression_in_query.sh index caa74523bd7..2936568c991 100755 --- a/tests/queries/0_stateless/02024_compression_in_query.sh +++ b/tests/queries/0_stateless/02024_compression_in_query.sh @@ -55,8 +55,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_compression_keyword;" [ -e "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz ] && rm "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz # create files using compression method and without it to check that both queries work correct -${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;" -${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;" # check content of files cp ${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_to_decomp.gz @@ -68,7 +68,7 @@ gunzip ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp.gz cat ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp # create table to check inserts -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " DROP TABLE IF EXISTS test_compression_keyword; CREATE TABLE test_compression_keyword (text String) Engine=Memory; INSERT INTO TABLE test_compression_keyword FROM INFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated; diff --git a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh index f055ea304b2..efc19cad054 100755 --- a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh +++ b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh @@ -17,7 +17,7 @@ echo -e "103" > "${CLICKHOUSE_TMP}"/test_infile_parallel_3 gzip "${CLICKHOUSE_TMP}"/test_infile_parallel -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -q "27" && echo "Correct" || echo 'Fail' -${CLICKHOUSE_LOCAL} --multiquery <&1 | grep 'AlterCommand' + $CLICKHOUSE_CLIENT --readonly 1 2>&1 | grep 'AlterCommand' diff --git a/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh b/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh index 59ede739e4a..3b71c8754c9 100755 --- a/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh +++ b/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --echo --multiline --multiquery -q """ +$CLICKHOUSE_LOCAL --echo --multiline -q """ SHOW TABLES; CREATE DATABASE test1; CREATE TABLE test1.table1 (a Int32) ENGINE=Memory; diff --git a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh index d0e61541b15..18ae2d7b4b3 100755 --- a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh +++ b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh @@ -10,7 +10,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do echo "Using storage policy: $STORAGE_POLICY" - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SET max_memory_usage='20G'; SET enable_filesystem_cache_on_write_operations = 0; @@ -25,7 +25,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1" 2>&1) - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -40,14 +40,14 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do LIMIT 1; """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ set remote_filesystem_read_method = 'read'; set local_filesystem_read_method = 'pread'; """ query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1" 2>&1) - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -63,13 +63,13 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ set remote_filesystem_read_method='threadpool'; """ query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1") - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -84,7 +84,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do LIMIT 1; """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SELECT * FROM test_02226 WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; SET enable_filesystem_cache_on_write_operations = 1; diff --git a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh index 344452767cc..a3fe5f19de0 100755 --- a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh +++ b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh @@ -17,7 +17,7 @@ export CURR_DATABASE="test_01889_sqllite_${CLICKHOUSE_DATABASE}" DB_PATH=${USER_FILES_PATH}/${CURR_DATABASE}_db1 -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" DROP DATABASE IF EXISTS ${CURR_DATABASE}; CREATE DATABASE ${CURR_DATABASE} ENGINE = SQLite('${DB_PATH}'); SHOW TABLES FROM ${CURR_DATABASE}; @@ -25,6 +25,6 @@ SHOW TABLES FROM ${CURR_DATABASE}; sqlite3 "${DB_PATH}" 'CREATE TABLE table1 (col1 text, col2 smallint);' -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" SHOW TABLES FROM ${CURR_DATABASE}; """ diff --git a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh index ffc38c0c1bd..aa5db33417c 100755 --- a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh +++ b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_suspicious_low_cardinality_types=1 --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --allow_suspicious_low_cardinality_types=1 --multiline --query=""" DROP TABLE IF EXISTS t_01411; DROP TABLE IF EXISTS t_01411_num; @@ -35,7 +35,7 @@ insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, n function go() { -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" select sum(toUInt64(str)), sum(toUInt64(pat)) from lc_dict_reading where val < 8129 or val > 8192 * 4; @@ -67,7 +67,7 @@ for _ in `seq 1 32`; do go | grep -q "Exception" && echo 'FAIL' || echo 'OK' ||: wait -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" DROP TABLE IF EXISTS t_01411; DROP TABLE IF EXISTS t_01411_num; """ diff --git a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh index 637e01b9e63..e5e717d00a8 100755 --- a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh +++ b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh @@ -15,7 +15,7 @@ SCHEMADIR=$CURDIR/format_schemas set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <&1 | rg -Fc "'w' character" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'w' character" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": wtf}'); END -$CLICKHOUSE_LOCAL --multiquery <&1 | rg -Fc "expected 'false'" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "expected 'false'" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": ftw}'); END -$CLICKHOUSE_LOCAL --multiquery <&1 | rg -Fc "'{' character" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'{' character" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": {}}'); END diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 2d0ff256c95..fa23d847d90 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -30,7 +30,7 @@ cp ${user_files_tmp_dir}/tmp_numbers_1.csv ${user_files_tmp_dir}/tmp/tmp_numbers ################# echo "Test 1: create filesystem database and check implicit calls" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; CREATE DATABASE test1 ENGINE = Filesystem; """ @@ -57,20 +57,20 @@ ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --multiline --multiquery --query """ +${CLICKHOUSE_CLIENT} --multiline --query """ USE test1; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; """ 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: path should be inside user_files -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 80b47282146..cc7f012c8cf 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ################# echo "Test 1: select from s3" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; CREATE DATABASE test1 ENGINE = S3; USE test1; @@ -17,7 +17,7 @@ SELECT * FROM \"http://localhost:11111/test/a.tsv\" ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1 # check credentials with absolute path -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = S3('', 'test', 'testtest'); USE test2; @@ -25,7 +25,7 @@ SELECT * FROM \"http://localhost:11111/test/b.tsv\" """ # check credentials with relative path -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = S3('http://localhost:11111/test', 'test', 'testtest'); USE test4; @@ -33,7 +33,7 @@ SELECT * FROM \"b.tsv\" """ # Check named collection loading -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test5; CREATE DATABASE test5 ENGINE = S3(s3_conn_db); SELECT * FROM test5.\`b.tsv\` @@ -41,20 +41,20 @@ SELECT * FROM test5.\`b.tsv\` ################# echo "Test 2: check exceptions" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "S3_ERROR" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ USE test3; SELECT * FROM \"abacaba\" """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Cleanup -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; DROP DATABASE IF EXISTS test2; DROP DATABASE IF EXISTS test3; diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 1eb22976b84..7fd35c72ef1 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -25,7 +25,7 @@ fi echo "Test 1: select from hdfs database" # Database without specific host -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_1; CREATE DATABASE test_hdfs_1 ENGINE = HDFS; USE test_hdfs_1; @@ -34,7 +34,7 @@ SELECT * FROM \"hdfs://localhost:12222/test_02725_1.tsv\" ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test_hdfs_1 # Database with host -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_2; CREATE DATABASE test_hdfs_2 ENGINE = HDFS('hdfs://localhost:12222'); USE test_hdfs_2; @@ -45,12 +45,12 @@ ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test_hdfs_2 ################# echo "Test 2: check exceptions" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_3; CREATE DATABASE test_hdfs_3 ENGINE = HDFS('abacaba'); """ 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_4; CREATE DATABASE test_hdfs_4 ENGINE = HDFS; USE test_hdfs_4; @@ -64,7 +64,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222 # Cleanup -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_1; DROP DATABASE IF EXISTS test_hdfs_2; DROP DATABASE IF EXISTS test_hdfs_3; diff --git a/tests/queries/0_stateless/02725_local_query_parameters.sh b/tests/queries/0_stateless/02725_local_query_parameters.sh index 92d7f645454..151a854d5b9 100755 --- a/tests/queries/0_stateless/02725_local_query_parameters.sh +++ b/tests/queries/0_stateless/02725_local_query_parameters.sh @@ -5,6 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --multiquery -q " +$CLICKHOUSE_LOCAL -q " SET param_x=1; SELECT {x:UInt64}, {x:String};" diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 2e55712e49c..f02e9bab2cd 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -5,8 +5,6 @@ Syntax error Empty query Empty query -BAD_ARGUMENTS -BAD_ARGUMENTS 301 302 304 diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index 7b959a3c3dc..4021194656b 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -4,18 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --multiquery "SELECT 100" -$CLICKHOUSE_LOCAL --multiquery "SELECT 101;" -$CLICKHOUSE_LOCAL --multiquery "SELECT 102;SELECT 103;" +$CLICKHOUSE_LOCAL "SELECT 100" +$CLICKHOUSE_LOCAL "SELECT 101;" +$CLICKHOUSE_LOCAL "SELECT 102;SELECT 103;" # Invalid SQL. -$CLICKHOUSE_LOCAL --multiquery "SELECT 200; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_LOCAL --multiquery "; SELECT 201;" 2>&1 | grep -o 'Empty query' -$CLICKHOUSE_LOCAL --multiquery "; S; SELECT 202" 2>&1 | grep -o 'Empty query' - -# Simultaneously passing --queries-file + --query (multiquery) is prohibited. -$CLICKHOUSE_LOCAL --queries-file "queries.csv" --multiquery "SELECT 250;" 2>&1 | grep -o 'BAD_ARGUMENTS' -$CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL "SELECT 200; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_LOCAL "; SELECT 201;" 2>&1 | grep -o 'Empty query' +$CLICKHOUSE_LOCAL "; S; SELECT 202" 2>&1 | grep -o 'Empty query' # Error expectation cases. # -n is also interpreted as a query diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh index 68c55f9b66a..18ffc9dfec3 100755 --- a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --query "SHOW TABLES" || echo "Failed" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables WHERE database = currentDatabase() FORMAT Null" || echo "Failed" # Multi queries are ok: -$CLICKHOUSE_LOCAL --multiquery "SELECT 1; SELECT 2;" || echo "Failed" +$CLICKHOUSE_LOCAL "SELECT 1; SELECT 2;" || echo "Failed" # It can run in interactive mode: function run() diff --git a/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh b/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh index d806b678456..9207e48092f 100755 --- a/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh +++ b/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_DIR=$CUR_DIR/data_tsv -$CLICKHOUSE_LOCAL --multiquery \ +$CLICKHOUSE_LOCAL \ "CREATE VIEW users AS SELECT * FROM file('$DATA_DIR/mock_data.tsv', TSVWithNamesAndTypes); CREATE TABLE users_output (name String, tag UInt64)ENGINE = Memory; INSERT INTO users_output WITH (SELECT groupUniqArrayArray(mapKeys(Tags)) FROM users) AS unique_tags SELECT UserName AS name, length(unique_tags) AS tag FROM users; diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index 08313e2fd3b..229f832ba14 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -39,7 +39,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # Then try to restore with the setting `restore_broken_parts_as_detached` set to true. $CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}') SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}' -$CLICKHOUSE_CLIENT --multiquery <&1 | grep -c "INVOKER") >= 1 )) && echo "OK" || echo "UNEXPECTED" (( $(${CLICKHOUSE_CLIENT} --query "SHOW TABLE $db.test_view_2" 2>&1 | grep -c "DEFINER = $user1") >= 1 )) && echo "OK" || echo "UNEXPECTED" -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" (( $(${CLICKHOUSE_CLIENT} --query "INSERT INTO $db.test_table VALUES ('foo'), ('bar');" 2>&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" -${CLICKHOUSE_CLIENT} --multiquery <\n" $CLICKHOUSE_CLIENT --query "SELECT * FROM file(${UNIX_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32');" -$CLICKHOUSE_CLIENT --multiquery --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32'); --{serverError 117}" +$CLICKHOUSE_CLIENT --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32'); --{serverError 117}" echo -e "\n<-- Read DOS endings with setting input_format_tsv_crlf_end_of_line=1 -->\n" $CLICKHOUSE_CLIENT --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32') SETTINGS input_format_tsv_crlf_end_of_line = 1;" diff --git a/tests/queries/0_stateless/02995_forget_partition.sh b/tests/queries/0_stateless/02995_forget_partition.sh index c22f5829130..6fa0b96e90d 100755 --- a/tests/queries/0_stateless/02995_forget_partition.sh +++ b/tests/queries/0_stateless/02995_forget_partition.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ drop table if exists forget_partition; create table forget_partition @@ -31,7 +31,7 @@ alter table forget_partition drop partition '20240102'; # DROP PARTITION do not wait for a part to be removed from memory due to possible concurrent SELECTs, so we have to do wait manually here while [[ $(${CLICKHOUSE_CLIENT} -q "select count() from system.parts where database=currentDatabase() and table='forget_partition' and partition IN ('20240101', '20240102')") != 0 ]]; do sleep 1; done -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ set allow_unrestricted_reads_from_keeper=1; select '---before---'; diff --git a/tests/queries/0_stateless/02995_index_1.sh b/tests/queries/0_stateless/02995_index_1.sh index a5f1b30c2e8..128697fd0fe 100755 --- a/tests/queries/0_stateless/02995_index_1.sh +++ b/tests/queries/0_stateless/02995_index_1.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_10.sh b/tests/queries/0_stateless/02995_index_10.sh index d72c7c72705..c15ba00fd05 100755 --- a/tests/queries/0_stateless/02995_index_10.sh +++ b/tests/queries/0_stateless/02995_index_10.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_2.sh b/tests/queries/0_stateless/02995_index_2.sh index e7451c7ee4b..a32f5c511f8 100755 --- a/tests/queries/0_stateless/02995_index_2.sh +++ b/tests/queries/0_stateless/02995_index_2.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh index 506429e2696..9cc937391fc 100755 --- a/tests/queries/0_stateless/02995_index_3.sh +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_4.sh b/tests/queries/0_stateless/02995_index_4.sh index 1a0458728f9..e450997e48b 100755 --- a/tests/queries/0_stateless/02995_index_4.sh +++ b/tests/queries/0_stateless/02995_index_4.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_5.sh b/tests/queries/0_stateless/02995_index_5.sh index 60c12a8146d..80f75a532e3 100755 --- a/tests/queries/0_stateless/02995_index_5.sh +++ b/tests/queries/0_stateless/02995_index_5.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_6.sh b/tests/queries/0_stateless/02995_index_6.sh index 4936f73f36b..e90387c7c0c 100755 --- a/tests/queries/0_stateless/02995_index_6.sh +++ b/tests/queries/0_stateless/02995_index_6.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_7.sh b/tests/queries/0_stateless/02995_index_7.sh index 26be310abce..a5fdd98b2f8 100755 --- a/tests/queries/0_stateless/02995_index_7.sh +++ b/tests/queries/0_stateless/02995_index_7.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_8.sh b/tests/queries/0_stateless/02995_index_8.sh index 8c2620b59fd..adb835aedca 100755 --- a/tests/queries/0_stateless/02995_index_8.sh +++ b/tests/queries/0_stateless/02995_index_8.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_9.sh b/tests/queries/0_stateless/02995_index_9.sh index 76160c62aaa..4b78777cd2a 100755 --- a/tests/queries/0_stateless/02995_index_9.sh +++ b/tests/queries/0_stateless/02995_index_9.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02998_native_parquet_reader.sh b/tests/queries/0_stateless/02998_native_parquet_reader.sh index d6369c4921b..95b659815ed 100755 --- a/tests/queries/0_stateless/02998_native_parquet_reader.sh +++ b/tests/queries/0_stateless/02998_native_parquet_reader.sh @@ -208,4 +208,4 @@ CH_SCHEMA="\ QUERY="SELECT * from file('$PAR_PATH', 'Parquet', '$CH_SCHEMA')" # there may be more than on group in parquet files, unstable results may generated by multithreads -$CLICKHOUSE_LOCAL --multiquery --max_threads 1 --max_parsing_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY" +$CLICKHOUSE_LOCAL --max_threads 1 --max_parsing_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY" diff --git a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh index f857358a5ea..8c6aa70f14c 100755 --- a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh +++ b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh @@ -9,7 +9,7 @@ db="$CLICKHOUSE_DATABASE" db_2="${db}_2" backup_name="${db}_backup" -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS mv; CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; diff --git a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh index 2ec5832fac6..96cbd391a44 100755 --- a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh +++ b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS mv; CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; diff --git a/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh b/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh index 8c79dfdbafc..312fb03668c 100755 --- a/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh +++ b/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh @@ -8,7 +8,7 @@ db=${CLICKHOUSE_DATABASE} user1="user1_03006_${db}_$RANDOM" user2="user2_03006_${db}_$RANDOM" -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep --text -F -v "ASan doesn't fully support makecontext/swapcontext functions" -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE test; " diff --git a/tests/queries/0_stateless/03212_thousand_exceptions.sh b/tests/queries/0_stateless/03212_thousand_exceptions.sh index 0a6abf35c10..1237cbf537f 100755 --- a/tests/queries/0_stateless/03212_thousand_exceptions.sh +++ b/tests/queries/0_stateless/03212_thousand_exceptions.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # This should not be too slow, even under sanitizers. -yes "SELECT throwIf(1); SELECT '.' FORMAT Values;" | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery --ignore-error 2>/dev/null +yes "SELECT throwIf(1); SELECT '.' FORMAT Values;" | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error 2>/dev/null From 6372fdee6d344bd87d58ce89fa069b55750c9aba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 13:46:05 +0200 Subject: [PATCH 228/496] Update tests --- tests/queries/0_stateless/02995_index_3.sh | 2 +- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh index 506429e2696..219ae81154f 100755 --- a/tests/queries/0_stateless/02995_index_3.sh +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..a927e4f1e1f 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-distributed-cache DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From eb129b539fce2a407182d892ce3bd00f782a5833 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 13:46:27 +0200 Subject: [PATCH 229/496] Add tests --- src/Storages/StorageKeeperMap.cpp | 135 +++++++++++++----- src/Storages/StorageKeeperMap.h | 3 +- .../test_keeper_map_retries/__init__.py | 0 .../configs/enable_keeper_map.xml | 3 + .../configs/fault_injection.xml | 6 + .../test_keeper_map_retries/test.py | 78 ++++++++++ .../02911_backup_restore_keeper_map.sh | 15 +- 7 files changed, 194 insertions(+), 46 deletions(-) create mode 100644 tests/integration/test_keeper_map_retries/__init__.py create mode 100644 tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml create mode 100644 tests/integration/test_keeper_map_retries/configs/fault_injection.xml create mode 100644 tests/integration/test_keeper_map_retries/test.py diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 5534bb7f346..09c21ae28f5 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -183,7 +183,7 @@ public: settings.insert_keeper_retry_max_backoff_ms}, context->getProcessListElement()}; - retries_ctl.retryLoop([&]() + zk_retry.retryLoop([&]() { auto zookeeper = storage.getClient(); auto keys_limit = storage.keysLimit(); @@ -205,12 +205,12 @@ public: for (const auto & [key, _] : new_values) key_paths.push_back(storage.fullPathForKey(key)); - zkutil::ZooKeeper::MultiExistsResponse results; + zkutil::ZooKeeper::MultiTryGetResponse results; if constexpr (!for_update) { if (!strict) - results = zookeeper->exists(key_paths); + results = zookeeper->tryGet(key_paths); } Coordination::Requests requests; @@ -231,7 +231,8 @@ public: { if (!strict && results[i].error == Coordination::Error::ZOK) { - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + if (results[i].data != new_values[key]) + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); } else { @@ -241,6 +242,9 @@ public: } } + if (requests.empty()) + return; + if (new_keys_num != 0) { auto will_be = current_keys_num + new_keys_num; @@ -259,7 +263,7 @@ public: }; template -class StorageKeeperMapSource : public ISource +class StorageKeeperMapSource : public ISource, WithContext { const StorageKeeperMap & storage; size_t max_block_size; @@ -290,8 +294,15 @@ public: KeyContainerPtr container_, KeyContainerIter begin_, KeyContainerIter end_, - bool with_version_column_) - : ISource(getHeader(header, with_version_column_)), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + bool with_version_column_, + ContextPtr context_) + : ISource(getHeader(header, with_version_column_)) + , WithContext(std::move(context_)) + , storage(storage_) + , max_block_size(max_block_size_) + , container(std::move(container_)) + , it(begin_) + , end(end_) , with_version_column(with_version_column_) { } @@ -316,12 +327,12 @@ public: for (auto & raw_key : raw_keys) raw_key = base64Encode(raw_key, /* url_encoding */ true); - return storage.getBySerializedKeys(raw_keys, nullptr, with_version_column); + return storage.getBySerializedKeys(raw_keys, nullptr, with_version_column, getContext()); } else { size_t elem_num = std::min(max_block_size, static_cast(end - it)); - auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr, with_version_column); + auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr, with_version_column, getContext()); it += elem_num; return chunk; } @@ -553,14 +564,31 @@ Pipe StorageKeeperMap::read( using KeyContainer = typename KeyContainerPtr::element_type; pipes.emplace_back(std::make_shared>( - *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column)); + *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column, context_)); } return Pipe::unitePipes(std::move(pipes)); }; - auto client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(zk_data_path))); + { + const auto & settings = context_->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + context_->getProcessListElement()}; + + std::vector children; + zk_retry.retryLoop([&] + { + auto client = getClient(); + children = client->getChildren(zk_data_path); + }); + return process_keys(std::make_shared>(std::move(children))); + } return process_keys(std::move(filtered_keys)); } @@ -571,11 +599,24 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); } -void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { checkTable(); - auto client = getClient(); - client->tryRemoveChildrenRecursive(zk_data_path, true); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + zk_retry.retryLoop([&] + { + auto client = getClient(); + client->tryRemoveChildrenRecursive(zk_data_path, true); + }); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) @@ -1064,10 +1105,11 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD if (raw_keys.size() != keys[0].column->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); - return getBySerializedKeys(raw_keys, &null_map, /* version_column */ false); + return getBySerializedKeys(raw_keys, &null_map, /* version_column */ false, getContext()); } -Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map, bool with_version) const +Chunk StorageKeeperMap::getBySerializedKeys( + const std::span keys, PaddedPODArray * null_map, bool with_version, const ContextPtr & local_context) const { Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); MutableColumns columns = sample_block.cloneEmptyColumns(); @@ -1084,17 +1126,27 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k null_map->resize_fill(keys.size(), 1); } - auto client = getClient(); - Strings full_key_paths; full_key_paths.reserve(keys.size()); for (const auto & key : keys) - { full_key_paths.emplace_back(fullPathForKey(key)); - } - auto values = client->tryGet(full_key_paths); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + zkutil::ZooKeeper::MultiTryGetResponse values; + zk_retry.retryLoop([&]{ + auto client = getClient(); + values = client->tryGet(full_key_paths); + }); for (size_t i = 0; i < keys.size(); ++i) { @@ -1182,16 +1234,16 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca if (commands.front().type == MutationCommand::Type::DELETE) { - MutationsInterpreter::Settings settings(true); - settings.return_all_columns = true; - settings.return_mutated_rows = true; + MutationsInterpreter::Settings mutation_settings(true); + mutation_settings.return_all_columns = true; + mutation_settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, local_context, - settings); + mutation_settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -1200,8 +1252,6 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca auto primary_key_pos = header.getPositionByName(primary_key); auto version_position = header.getPositionByName(std::string{version_column_name}); - auto client = getClient(); - Block block; while (executor.pull(block)) { @@ -1229,7 +1279,23 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca } Coordination::Responses responses; - auto status = client->tryMulti(delete_requests, responses, /* check_session_valid */ true); + + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + Coordination::Error status; + zk_retry.retryLoop([&] + { + auto client = getClient(); + status = client->tryMulti(delete_requests, responses, /* check_session_valid */ true); + }); if (status == Coordination::Error::ZOK) return; @@ -1241,9 +1307,14 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca for (const auto & delete_request : delete_requests) { - auto code = client->tryRemove(delete_request->getPath()); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException::fromPath(code, delete_request->getPath()); + zk_retry.retryLoop([&] + { + auto client = getClient(); + status = client->tryRemove(delete_request->getPath()); + }); + + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNONODE) + throw zkutil::KeeperException::fromPath(status, delete_request->getPath()); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index d4556792c48..cfbb35ab2fe 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -54,7 +54,8 @@ public: Names getPrimaryKey() const override { return {primary_key}; } Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const override; - Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map, bool with_version) const; + Chunk getBySerializedKeys( + std::span keys, PaddedPODArray * null_map, bool with_version, const ContextPtr & local_context) const; Block getSampleBlock(const Names &) const override; diff --git a/tests/integration/test_keeper_map_retries/__init__.py b/tests/integration/test_keeper_map_retries/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml b/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml new file mode 100644 index 00000000000..145945c7c7c --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -0,0 +1,6 @@ + + + 0.05 + 0.05 + + diff --git a/tests/integration/test_keeper_map_retries/test.py b/tests/integration/test_keeper_map_retries/test.py new file mode 100644 index 00000000000..352119147cd --- /dev/null +++ b/tests/integration/test_keeper_map_retries/test.py @@ -0,0 +1,78 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +import os + +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper_map.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +def start_clean_clickhouse(): + # remove fault injection if present + if "fault_injection.xml" in node.exec_in_container( + ["bash", "-c", "ls /etc/clickhouse-server/config.d"] + ): + print("Removing fault injection") + node.exec_in_container( + ["bash", "-c", "rm /etc/clickhouse-server/config.d/fault_injection.xml"] + ) + node.restart_clickhouse() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def repeat_query(query, repeat): + for _ in range(repeat): + node.query( + query, + settings={ + "keeper_max_retries": 20, + "keeper_retry_max_backoff_ms": 10000, + }, + ) + + +def test_queries(started_cluster): + start_clean_clickhouse() + + node.query("DROP TABLE IF EXISTS keeper_map_retries SYNC") + node.query( + "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a" + ) + + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CONFIG_DIR, "fault_injection.xml"), + "/etc/clickhouse-server/config.d/fault_injection.xml", + ) + node.start_clickhouse() + + repeat_count = 10 + + repeat_query( + "INSERT INTO keeper_map_retries SELECT number, number FROM numbers(500)", + repeat_count, + ) + repeat_query("SELECT * FROM keeper_map_retries", repeat_count) + repeat_query( + "ALTER TABLE keeper_map_retries UPDATE b = 3 WHERE a > 2", repeat_count + ) + repeat_query("ALTER TABLE keeper_map_retries DELETE WHERE a > 2", repeat_count) + repeat_query("TRUNCATE keeper_map_retries", repeat_count) diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index ee070b40f6f..c04667505c3 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -13,20 +13,9 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; " -# KeeperMap table engine doesn't have internal retries for interaction with Keeper. Do it on our own, otherwise tests with overloaded server can be flaky. -while true -do - $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; - " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue - break -done +$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" -while true -do - $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; - " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue - break -done +$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" backup_path="$database_name" for i in $(seq 1 3); do From a1ececb24c0a6f21d3985f5a9f8a726befef78c3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 11:51:48 +0000 Subject: [PATCH 230/496] Fix use-of-unitialized-value --- src/Columns/ColumnAggregateFunction.cpp | 33 ++++++++++++++++++++++--- 1 file changed, 30 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 9934970c868..4bc48c62eb4 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -332,9 +332,36 @@ void ColumnAggregateFunction::expand(const Filter & mask, bool inverted) { ensureOwnership(); Arena & arena = createOrGetArena(); - char * default_ptr = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); - func->create(default_ptr); - expandDataByMask(data, mask, inverted, default_ptr); + + if (mask.size() < data.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mask size should be no less than data size."); + + ssize_t from = data.size() - 1; + ssize_t index = mask.size() - 1; + data.resize(mask.size()); + while (index >= 0) + { + if (!!mask[index] ^ inverted) + { + if (from < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too many bytes in mask"); + + /// Copy only if it makes sense. + if (index != from) + data[index] = data[from]; + --from; + } + else + { + data[index] = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); + func->create(data[index]); + } + + --index; + } + + if (from != -1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough bytes in mask"); } ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const From fd075470d6e141d5aa4d01ccefcc3d3ee04130c7 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 11:54:00 +0000 Subject: [PATCH 231/496] Add docs --- tests/integration/README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/integration/README.md b/tests/integration/README.md index cde4cb05aec..d5137a9c148 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -130,6 +130,14 @@ docker build -t clickhouse/integration-test . ``` The helper container used by the `runner` script is in `docker/test/integration/runner/Dockerfile`. +It can be rebuild with + +``` +cd docker/test/integration/runner +docker build -t clickhouse/integration-test-runner . +``` + +Also you need to add option --network=host if you rebuild image for a local integration testsing. ### Adding new tests From fd26672864a7e1557908b878d7daa018de20c61a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 13:54:30 +0200 Subject: [PATCH 232/496] Revert some change --- src/Storages/StorageKeeperMap.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 09c21ae28f5..1559b442e43 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -35,7 +35,6 @@ #include #include -#include "Common/ZooKeeper/ZooKeeperRetries.h" #include #include #include @@ -44,6 +43,7 @@ #include #include #include +#include #include #include @@ -205,12 +205,12 @@ public: for (const auto & [key, _] : new_values) key_paths.push_back(storage.fullPathForKey(key)); - zkutil::ZooKeeper::MultiTryGetResponse results; + zkutil::ZooKeeper::MultiExistsResponse results; if constexpr (!for_update) { if (!strict) - results = zookeeper->tryGet(key_paths); + results = zookeeper->exists(key_paths); } Coordination::Requests requests; @@ -231,8 +231,7 @@ public: { if (!strict && results[i].error == Coordination::Error::ZOK) { - if (results[i].data != new_values[key]) - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); } else { @@ -242,9 +241,6 @@ public: } } - if (requests.empty()) - return; - if (new_keys_num != 0) { auto will_be = current_keys_num + new_keys_num; From 0124d211ec81a3779fe0e99c868fe85be8856629 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 14:00:52 +0200 Subject: [PATCH 233/496] Better --- tests/integration/test_replicated_table_attach/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 499220def2c..de60b7ec291 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -41,6 +41,7 @@ def start_clean_clickhouse(): def test_startup_with_small_bg_pool(started_cluster): start_clean_clickhouse() + node.query("DROP TABLE IF EXISTS replicated_table SYNC") node.query( "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" ) @@ -54,11 +55,10 @@ def test_startup_with_small_bg_pool(started_cluster): node.restart_clickhouse(stop_start_wait_sec=10) assert_values() - node.query("DROP TABLE replicated_table SYNC") - def test_startup_with_small_bg_pool_partitioned(started_cluster): start_clean_clickhouse() + node.query("DROP TABLE IF EXISTS replicated_table_partitioned SYNC") node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" ) @@ -81,5 +81,3 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): # check that we activate it in the end node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") - - node.query("DROP TABLE replicated_table_partitioned SYNC") From de99ee1b05e68b964535664d4197afd0944d0261 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 12:07:31 +0000 Subject: [PATCH 234/496] Change docs --- tests/integration/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index d5137a9c148..c1eb511fa44 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -137,7 +137,7 @@ cd docker/test/integration/runner docker build -t clickhouse/integration-test-runner . ``` -Also you need to add option --network=host if you rebuild image for a local integration testsing. +If your docker configuration doesn't allow access to public internet with docker build command you may also need to add option --network=host if you rebuild image for a local integration testsing. ### Adding new tests From c3e8825c8f14bb82f60c41754a021813d3dbc8aa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 14:23:32 +0200 Subject: [PATCH 235/496] Use correct order of fields in StorageURLSource --- src/Storages/StorageURL.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 63d01a02417..6c95cad474c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -228,12 +228,12 @@ private: bool need_only_count; size_t total_rows_in_file = 0; + Poco::Net::HTTPBasicCredentials credentials; + std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; - - Poco::Net::HTTPBasicCredentials credentials; }; class StorageURLSink : public SinkToStorage From 8b52d7b711d54f1d4bb5b2f39bf4aea3966f64dc Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 13:35:19 +0100 Subject: [PATCH 236/496] fxs --- src/IO/S3/Client.cpp | 19 +++++++++---------- src/IO/S3/Client.h | 2 +- .../test_checking_s3_blobs_paranoid/test.py | 12 ++++++++++++ tests/integration/test_storage_delta/test.py | 12 ++++++++++++ 4 files changed, 34 insertions(+), 11 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7196dfa9bdc..8f037ea71be 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -385,10 +385,9 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - if (isClientForDisk() && error.GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - - return enrichErrorMessage( + /// The next call is NOT a recurcive call + /// This is a virtuall call Aws::S3::S3Client::HeadObject(const Model::HeadObjectRequest&) + return processRequestResult( HeadObject(static_cast(request))); } @@ -409,11 +408,8 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { - auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); - if (!resp.IsSuccess() && isClientForDisk() && resp.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - - return enrichErrorMessage(std::move(resp)); + return processRequestResult( + doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); })); } Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const @@ -699,11 +695,14 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request } template -RequestResult Client::enrichErrorMessage(RequestResult && outcome) const +RequestResult Client::processRequestResult(RequestResult && outcome) const { if (outcome.IsSuccess() || !isClientForDisk()) return std::forward(outcome); + if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + String enriched_message = fmt::format( "{} {}", outcome.GetError().GetMessage(), diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 11cace4e1fd..e54953419e1 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -274,7 +274,7 @@ private: void insertRegionOverride(const std::string & bucket, const std::string & region) const; template - RequestResult enrichErrorMessage(RequestResult && outcome) const; + RequestResult processRequestResult(RequestResult && outcome) const; String initial_endpoint; std::shared_ptr credentials_provider; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 1ed70e20b79..dde636b5d29 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -689,3 +689,15 @@ def test_no_key_found_disk(cluster, broken_s3): "DB::Exception: The specified key does not exist. This error happened for S3 disk." in error ) + + s3_disk_no_key_errors_metric_value = int( + node.query( + """ + SELECT value + FROM system.metrics + WHERE metric = 'S3DiskNoKeyErrors' + """ + ).strip() + ) + + assert s3_disk_no_key_errors_metric_value > 0 diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d3dd7cfe52a..67cc7cdd6da 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -452,6 +452,18 @@ def test_restart_broken(started_cluster): f"SELECT count() FROM {TABLE_NAME}" ) + s3_disk_no_key_errors_metric_value = int( + instance.query( + """ + SELECT value + FROM system.metrics + WHERE metric = 'S3DiskNoKeyErrors' + """ + ).strip() + ) + + assert s3_disk_no_key_errors_metric_value == 0 + minio_client.make_bucket(bucket) upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") From d69f6cccde7633214eba48c08d7647e4ea9a40da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 15:08:26 +0200 Subject: [PATCH 237/496] Fix --- src/Common/ThreadPoolTaskTracker.cpp | 5 ++++- src/IO/WriteBufferFromS3.cpp | 9 ++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/ThreadPoolTaskTracker.cpp b/src/Common/ThreadPoolTaskTracker.cpp index 61d34801f7a..1697a13f780 100644 --- a/src/Common/ThreadPoolTaskTracker.cpp +++ b/src/Common/ThreadPoolTaskTracker.cpp @@ -19,6 +19,10 @@ TaskTracker::TaskTracker(ThreadPoolCallbackRunnerUnsafe scheduler_, size_t TaskTracker::~TaskTracker() { + /// Tasks should be waited outside of dtor. + /// Important for WriteBufferFromS3/AzureBlobStorage, where TaskTracker is currently used. + chassert(finished_futures.empty() && futures.empty()); + safeWaitAll(); } @@ -170,4 +174,3 @@ bool TaskTracker::isAsync() const } } - diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 3682e49b018..e702b4d35ad 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -277,12 +277,10 @@ WriteBufferFromS3::~WriteBufferFromS3() "The file might not be written to S3. " "{}.", getVerboseLogDetails()); - return; } - - /// That destructor could be call with finalized=false in case of exceptions - if (!finalized && !canceled) + else if (!finalized) { + /// That destructor could be call with finalized=false in case of exceptions LOG_INFO( log, "WriteBufferFromS3 is not finalized in destructor. " @@ -291,9 +289,10 @@ WriteBufferFromS3::~WriteBufferFromS3() getVerboseLogDetails()); } + /// Wait for all tasks, because they contain reference to this write buffer. task_tracker->safeWaitAll(); - if (!multipart_upload_id.empty() && !multipart_upload_finished) + if (!canceled && !multipart_upload_id.empty() && !multipart_upload_finished) { LOG_WARNING(log, "WriteBufferFromS3 was neither finished nor aborted, try to abort upload in destructor. {}.", getVerboseLogDetails()); tryToAbortMultipartUpload(); From 20faed85ca30c6352fd091e8d4d763fb98fe1311 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 15:18:37 +0200 Subject: [PATCH 238/496] Remove useless file --- tests/queries/0_stateless/data_minio/test :: 03215_archive.csv | 1 - 1 file changed, 1 deletion(-) delete mode 100644 tests/queries/0_stateless/data_minio/test :: 03215_archive.csv diff --git a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv +++ /dev/null @@ -1 +0,0 @@ -1 From 21aa514c80cb463f079f9877ae97048a8b13dfbe Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jul 2024 15:03:09 +0100 Subject: [PATCH 239/496] don't run removed tests --- tests/performance/scripts/entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/scripts/entrypoint.sh b/tests/performance/scripts/entrypoint.sh index 0c3bfa550f4..db7d96ad150 100755 --- a/tests/performance/scripts/entrypoint.sh +++ b/tests/performance/scripts/entrypoint.sh @@ -118,7 +118,7 @@ then # far in the future and have unrelated test changes. base=$(git -C right/ch merge-base pr origin/master) git -C right/ch diff --name-only "$base" pr -- . | tee all-changed-files.txt - git -C right/ch diff --name-only "$base" pr -- tests/performance/*.xml | tee changed-test-definitions.txt + git -C right/ch diff --name-only --diff-filter=d "$base" pr -- tests/performance/*.xml | tee changed-test-definitions.txt git -C right/ch diff --name-only "$base" pr -- :!tests/performance/*.xml :!docker/test/performance-comparison | tee other-changed-files.txt fi From 7d5c30e76cf0fd17515803fec96899f4aad1294e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 16:26:19 +0200 Subject: [PATCH 240/496] No retries when partitioned --- tests/integration/test_keeper_map/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 31316af7b1e..7aee5df5746 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -46,7 +46,11 @@ def assert_keeper_exception_after_partition(query): with PartitionManager() as pm: pm.drop_instance_zk_connections(node) try: - error = node.query_and_get_error_with_retry(query, sleep_time=1) + error = node.query_and_get_error_with_retry( + query, + sleep_time=1, + settings={"insert_keeper_max_retries": 1, "keeper_max_retries": 1}, + ) assert "Coordination::Exception" in error except: print_iptables_rules() @@ -84,7 +88,9 @@ def test_keeper_map_without_zk(started_cluster): node.restart_clickhouse(60) try: error = node.query_and_get_error_with_retry( - "SELECT * FROM test_keeper_map_without_zk", sleep_time=1 + "SELECT * FROM test_keeper_map_without_zk", + sleep_time=1, + settings={"keeper_max_retries": 1}, ) assert "Failed to activate table because of connection issues" in error except: From 83c6d97cd2aae0a3f79c2776ffc3a5691f8fd4bb Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Tue, 30 Jul 2024 17:34:38 +0200 Subject: [PATCH 241/496] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 20353 -> 18666 bytes tests/queries/0_stateless/00309_formats.sql | 5 +++++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index cab311692be229716b58af39079275d3942b01cc..f3ea45520bb50fb936caf6724e9fedf3cdd00b75 100644 GIT binary patch delta 3052 zcmeIy$4}c(00wZIgtksdZ-w^I3JD}eL2?j6{R4XG1$t-?y;Q0c4wb5?2YL>Bj|8*# z-Xp>6y?59U_TGDMNaN%a?SDX3#j^bK`<|bk?ZfAlN^UDiNE{JVHgaD5e8+19DxExLIy^` zXcz-yVH}Ky2`~{R!DN^MQ}xOSA`L-0%z&9N3ueO{m<#h@J}iKRum~2z5?IQq+n-P( zP{A_D!g5#vD`6F^hBdGj*1>w%02^VGPHumqnY-wY76U{pY=iBv19rkL*bRGNFYJT; zPz?v*AoeGQ5Ddc+I10z$IGlhQI0>iVG@OC6a1PGH1;h6f#3F(vxC~d|DqMr>a070_ zEw~ML;4a*Q`|v>j|8;{y1V``~p1@Oh2G8LIyo6Wq8s5NLcn9yHg%&qjk~HD*O`kk{ zMhS#wBF4@26;n)ok$HM~`}q3#2LuKMhlGZON5~bCOjLACY+QUoVp4KSYFc_mW>$7i zZeD&tVNr2OsZv$OmRD3(RoB$k)i*RYHMg|3wRd!Ob@%l4^{WR4hlWQ+$Hpf#lT*_( zvvczci%ZKZt842Un_JsEyL2uP zs~KC(*lPbltFhlLRQ$(oSyIc$t;|{F>ph;XVgHfpC$+5H%8ajnp%+Y<&DY2OORU_k z(V7&oZmv$Gj7LgIiK)l0pFiA9KYx&VyyL(B=HYJsi3@C{mioxSQew%acGgGsx>HI? tDE$HfH|yd|adXt)D_oo(mI%Z&HeM!>@7a6Ht;zJ#Jk0d+*gt^JbdYKDu4%*X9d$a^ZOYUl#FBHF2yr}q<;@67HiYtm^#Z|>iir*-% zDXuF{6fY}&tN5Mb_lj2(e^C5U@h8Qria#snd+xpO{s$g>=;22meeCfko}78A|MWA@J~z8>|MLgJ!58LIidkNiRoxsq zJUB8u`r@(UFP(V#6D$6wrgio0oJnBji=r;A-&v0K-- z%ZqMZ-|En<>suYVb$zQtx30$|*-G(uTzhnP-g&=^d;Asr39~@AuK#;~vDmn-A16x* zR`tnxBCGmdyM|AAwVs$&ebTP#ldV?ulqNAv(}O7;BueJfKp}=K;v>=KamanSU6UWrDN?_JXVk8gMI`72?`P<#P@i7&<`pS zWF+WF5R#xIK}v#_1ThI}667T4Nf4BvC_z$!rUX$5suE-+=t>Zlpe#XJg0=*43F;E$ zCFn~Kn4mC0VuHp5kqIgjWG3iL5SpMgL28251hENf6XYi7O%R-*I6-oP<^<6RsuN@< z=uQxxpgci(g7yUQ3F;H%C+JUb0Ko$U7Z7|va00;#1UC@;KyU=X69iWfd_iyq!5aj3 z5d48~#s?7ye?<5r!XFX-i10^*KO+1Q;g1M^M8t7~KO+1Q;g1M^MEE1Z9})hD9M2rk z64B_#G0j2EQO#k^am|4Ze?<5r!XFX-i0}tWF!-P!{1M@g2!BNQBf=jM{)q5Lgg+wu z5#f&re?<5r!XFX-i10^*KO+1Q;g1M^#G){NzzgHs7A`ETZCepq_933?i~U2rWWJXqI8KuOcyVRaKQz-H_k$cyu3VDiwye}g+f_bcJ1@D;TzuN5 zD@s+avnf(-W3}#V+EUgwouv-u47MnGiFGeI6M7J0X>pSftOBKaoh3fnm@GvczL`Ta z^L_N{6Fw0i&Ve~T1wJJ{6+SgSO&=q%!s~pgh07?nk!+)EGuh^{Eo58Dw&Hr*^k(>Z zybQ>^Hqtrc{32`%3@;;YAm1kG-$5QXD0j)XNqnD@?1}G`_*RMUmH1|f@0J>DqOcTU z*dnqtku4%yM7D@*5!oWLMaC8xTV$doTV!mJu|>uf8Czs*k+DV27CBqwY>|unY>~4? z&K5aasMw-ni;68Owy4;mVvC9`Dz>QDqGpSlEo!!?*`j8Pnk{O!sM(@si<&JOwrJR* zVT*<>8n$TIqG5}MEgH6H*a8(t*|H6)jlxDpGfP)fUuE8&1TrL@Sn5)r6Ya*MU7eL0wb!Wn76AR5ztX z#+6_}jZ<1=T!|M{I=Mw62Gvg4mvJR%Q1g@)8CT*4l}~AraV2z6|Kt`49@Ic(U&fXA zK_ygLWLya$)I+&Nf(SKH*_Ux8j!+qu78zGU3H4EFk#QxOP$lISi6>M_WnadXkV3sw zT4Y>_DpXCSg~!FbC|e#E)l=EBEh?z8Wm{BHWy|BDk}6xVZKpC4TZt>wRQasL73!+8 zFSZg_sIE$j*tQdl#8s5ILYJI}Nejl>nIuG~}N3YAyc7h8!d)L*4V z#+A526;@hgT!|~xV!1`)3iVjom;05tLRD5;5#VP`KA{>G~DrX+|J4NE4^tt78W7My1Ewbk!9(?&_}d~o*M=)&sS=v zm5YR#+gx;@nz8VMIrYYX7g?<00RC^)91khQIPJmG@OrgrFt^k;LYv_Lo5d@`!DxB3 zGDLsD_K3_;G)I{^%FR(>j!JV>nWNeqHPfRx<;2)BT_0Y1Xf&A9h!Y`)0|nvDH@x8n zHr{gEJg2=CS+6-2Y@=}a^V*~wFA#5d<+f6RtRKlbl5c8*WMo}Q)|X_RN!FWW-AUK{ zDc6tAH={hZdv7)Rc=}3mQ`+9<(5@$S>={hVObTHRbg2bmQ`U{6_yqMxorr-!m=tXtHQD> zEvwS9DlMzhvMMdB(y}TotJ1P6EvwS9sw}I@vZ^er%Cf2~tID#fEUU`0sw}I@vZ^hs z+On!GtJ<=vEvwqHsx7P9vZ^hs+OldatH!cwEUU({YAmb9vT7`=#}TLyo>A3TXJ|u*PXZY@b0cX?~)z)dEQU-ex~gv+M8t>k(P&*cKVkL~(9{aiZ0vklwdB?P?7?an*> zTu#9I-mbsX&!q*_8f<@;81VeWop<`V+<^DVU4N&aOAmPG-1T?*xeS5#(rtg2Bk;7w zop<`VG=X>8U4N&a%M^I;-Sv0+xnzOoK(@b27kFphop<`VjDh#+U4N&aOB#6h-u8EC z1J9b=d8ePt9C&Zv^>_NY_NY{DCT;>+ke)`2)`|ZGV?P@Y@r2-syKefBb6wVB0s=_`E`L+6m)Ezc_5xV{`a(e6A7cfwpXiGp`$XFl3xq_$>F>d{|a@uQ~ Lvprlx%YOd>&)TR8 diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index b0939c00a10..0366cdeea5c 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -9,3 +9,8 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; + +SET enable_named_columns_in_function_tuple = 1; + +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From aa26291ff25b16539efa3d50a540fc32a05b702d Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 18:15:24 +0100 Subject: [PATCH 242/496] fxs --- src/Databases/DatabaseReplicated.cpp | 57 +++++++++++++++---- src/Databases/DatabaseReplicatedWorker.cpp | 21 +++++-- src/Databases/DatabaseReplicatedWorker.h | 4 +- src/Storages/System/StorageSystemClusters.cpp | 6 +- .../test_recovery_time_metric/test.py | 36 ++++++++++-- 5 files changed, 98 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b11b9382732..06cea65d62e 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -340,31 +341,63 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const { - ReplicasInfo res; + Strings paths_get, paths_exists; + + paths_get.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); - auto zookeeper = getZooKeeper(); const auto & addresses_with_failover = cluster_->getShardsAddresses(); const auto & shards_info = cluster_->getShardsInfo(); - + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); + paths_exists.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); + } + } + try { - UInt32 max_log_ptr = parse(zookeeper->get(zookeeper_path + "/max_log_ptr")); + auto current_zookeeper = getZooKeeper(); + auto get_res = current_zookeeper->get(paths_get); + auto exist_res = current_zookeeper->exists(paths_exists); + chassert(get_res.size() == exist_res.size() + 1); + auto max_log_ptr_zk = get_res[0]; + if (max_log_ptr_zk.error != Coordination::Error::ZOK) + throw Coordination::Exception(max_log_ptr_zk.error); + + UInt32 max_log_ptr = parse(max_log_ptr_zk.data); + + ReplicasInfo replicas_info; + replicas_info.resize(exist_res.size()); + + size_t global_replica_index = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - UInt32 log_ptr = parse(zookeeper->get(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr")); - bool is_active = zookeeper->exists(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - res.push_back(ReplicaInfo{ - .is_active = is_active, - .replication_lag = max_log_ptr - log_ptr, + auto replica_active = exist_res[global_replica_index]; + auto replica_log_ptr = get_res[global_replica_index + 1]; + + if (replica_active.error != Coordination::Error::ZOK && replica_active.error != Coordination::Error::ZNONODE) + throw Coordination::Exception(replica_active.error); + + if (replica_log_ptr.error != Coordination::Error::ZOK) + throw Coordination::Exception(replica_log_ptr.error); + + replicas_info[global_replica_index] = ReplicaInfo{ + .is_active = replica_active.error == Coordination::Error::ZOK, + .replication_lag = max_log_ptr - parse(replica_log_ptr.data), .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, - }); + }; + + ++global_replica_index; } } - return res; + + return replicas_info; } catch (...) { tryLogCurrentException(log); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index a9a74c5f56a..4e7408aa96e 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,8 +32,11 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { - initialization_duration_timer.restart(); - initializing.store(true, std::memory_order_release); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.emplace(); + initialization_duration_timer->start(); + } while (!stop_flag) { @@ -72,7 +75,10 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - initializing.store(false, std::memory_order_relaxed); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } return true; } catch (...) @@ -82,7 +88,11 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - initializing.store(false, std::memory_order_relaxed); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } + return false; } @@ -466,7 +476,8 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const { - return initializing.load(std::memory_order_acquire) ? initialization_duration_timer.elapsedMilliseconds() : 0; + std::lock_guard lock(initialization_duration_timer_mutex); + return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; } } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 3e5887be825..2309c831839 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -59,8 +59,8 @@ private: /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; - Stopwatch initialization_duration_timer; - std::atomic initializing = false; + std::optional initialization_duration_timer; + mutable std::mutex initialization_duration_timer_mutex; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 0da4bd70cbd..4b9802c9089 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -71,7 +71,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); - size_t replica_idx = 0; + size_t global_replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { const auto & shard_info = shards_info[shard_index]; @@ -108,7 +108,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam } else { - const auto & replica_info = replicas_info[replica_idx++]; + const auto & replica_info = replicas_info[global_replica_idx]; res_columns[i++]->insert(replica_info.is_active); res_columns[i++]->insert(replica_info.replication_lag); if (replica_info.recovery_time != 0) @@ -116,6 +116,8 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam else res_columns[i++]->insertDefault(); } + + ++global_replica_idx; } } } diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 4dad844b950..8f369d7759c 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -5,7 +5,6 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.xml"], - with_zookeeper=True, stay_alive=True, ) @@ -21,9 +20,36 @@ def start_cluster(): def test_recovery_time_metric(start_cluster): node.query( - "CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');" + """ + CREATE DATABASE rdb + ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') + """ ) - node.query("CREATE TABLE rdb.t (x UInt32) ENGINE = MergeTree ORDER BY x;") - node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) + + node.query( + """ + CREATE TABLE rdb.t + ( + `x` UInt32 + ) + ENGINE = MergeTree + ORDER BY x + """ + ) + + node.exec_in_container( + ["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"] + ) + node.restart_clickhouse() - assert node.query("SELECT any(recovery_time) FROM system.clusters;") != "0\n" + + ret = int( + node.query( + """ + SELECT recovery_time + FROM system.clusters + WHERE cluster = 'rdb' + """ + ).strip() + ) + assert ret > 0 From 7e51e9962c34320a9c60ba6abcf8b38cf517e86c Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Tue, 30 Jul 2024 17:20:07 +0000 Subject: [PATCH 243/496] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22ff9b7194f..a8110500f13 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -341,15 +341,19 @@ void MergeTreeDeduplicationLog::shutdown() stopped = true; if (current_writer) { + /// If an error has occurred during finalize, we'd like to have the exception set for reset. + /// Otherwise, we'll be in a situation when a finalization didn't happen, and we didn't get + /// any error, causing logical error (see ~MemoryBuffer()). try { current_writer->finalize(); + current_writer.reset(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + current_writer.reset(); } - current_writer.reset(); } } From 51af0d305c9959fb4870bd8a57035d48207648b2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 30 Jul 2024 20:00:23 +0200 Subject: [PATCH 244/496] Reduce number of tested combinations --- tests/queries/0_stateless/02473_multistep_prewhere.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index a942568233c..11095202039 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -193,7 +193,7 @@ def main(): url = os.environ["CLICKHOUSE_URL"] + "&max_threads=1" default_index_granularity = 10 - total_rows = 8 * default_index_granularity + total_rows = 7 * default_index_granularity step = default_index_granularity session = requests.Session() for index_granularity in [ From dacf044c3dee65d799242b7f4846f7d6d8b2bd34 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 30 Jul 2024 19:20:52 +0000 Subject: [PATCH 245/496] Update version_date.tsv and changelogs after v24.7.1.2915-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.7.1.2915-stable.md | 524 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 530 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.7.1.2915-stable.md diff --git a/SECURITY.md b/SECURITY.md index 53328b6e16b..8930dc96f8a 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,9 +14,10 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.7 | ✔️ | | 24.6 | ✔️ | | 24.5 | ✔️ | -| 24.4 | ✔️ | +| 24.4 | ❌ | | 24.3 | ✔️ | | 24.2 | ❌ | | 24.1 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index c59ef1b919a..e99c86267f9 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.6.2.17" +ARG VERSION="24.7.1.2915" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 240df79aeb1..fb562b911a3 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.6.2.17" +ARG VERSION="24.7.1.2915" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index ac64655991a..51f4e6a0f40 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.6.2.17" +ARG VERSION="24.7.1.2915" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.7.1.2915-stable.md b/docs/changelogs/v24.7.1.2915-stable.md new file mode 100644 index 00000000000..abffbe58bfc --- /dev/null +++ b/docs/changelogs/v24.7.1.2915-stable.md @@ -0,0 +1,524 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.1.2915-stable (a37d2d43da7) FIXME as compared to v24.7.1.1-new (aa023477a92) + +#### Backward Incompatible Change +* Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). +* Forbid `CREATE MATERIALIZED VIEW ... ENGINE Replicated*MergeTree POPULATE AS SELECT ...` with Replicated databases. [#63963](https://github.com/ClickHouse/ClickHouse/pull/63963) ([vdimir](https://github.com/vdimir)). +* `clickhouse-keeper-client` will only accept paths in string literals, such as `ls '/hello/world'`, not bare strings such as `ls /hello/world`. [#65494](https://github.com/ClickHouse/ClickHouse/pull/65494) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. This fixes a typo reported in [#66179](https://github.com/ClickHouse/ClickHouse/issues/66179). [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Extend function `tuple` to construct named tuples in query. Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). +* `ASOF JOIN` support for `full_sorting_join` algorithm Close [#54493](https://github.com/ClickHouse/ClickHouse/issues/54493). [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). +* A new table function, `fuzzQuery,` was added. This function allows you to modify a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1');`. [#62103](https://github.com/ClickHouse/ClickHouse/pull/62103) ([pufit](https://github.com/pufit)). +* Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). +* Support JWT authentication in `clickhouse-client`. [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). +* Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). +* Support accept_invalid_certificate in client's config in order to allow for client to connect over secure TCP to a server running with self-signed certificate - can be used as a shorthand for corresponding `openSSL` client settings `verificationMode=none` + `invalidCertificateHandler.name=AcceptCertificateHandler`. [#65238](https://github.com/ClickHouse/ClickHouse/pull/65238) ([peacewalker122](https://github.com/peacewalker122)). +* Add system.error_log which contains history of error values from table system.errors, periodically flushed to disk. [#65381](https://github.com/ClickHouse/ClickHouse/pull/65381) ([Pablo Marcos](https://github.com/pamarcos)). +* Add aggregate function `groupConcat`. About the same as `arrayStringConcat( groupArray(column), ',')` Can receive 2 parameters: a string delimiter and the number of elements to be processed. [#65451](https://github.com/ClickHouse/ClickHouse/pull/65451) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Add AzureQueue storage. [#65458](https://github.com/ClickHouse/ClickHouse/pull/65458) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). +* Allow system administrators to configure `logger.console_log_level`. [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). +* Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). +* Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). + +#### Performance Improvement +* Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). +* Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). +* Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). +* Improve performance of loading `plain_rewritable` metadata. [#65634](https://github.com/ClickHouse/ClickHouse/pull/65634) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attaching tables on read-only disks will use fewer resources by not loading outdated parts. [#65635](https://github.com/ClickHouse/ClickHouse/pull/65635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). +* Unload primary index of outdated parts to reduce total memory usage. [#65852](https://github.com/ClickHouse/ClickHouse/pull/65852) ([Anton Popov](https://github.com/CurtizJ)). +* Functions `replaceRegexpAll` and `replaceRegexpOne` are now significantly faster if the pattern is trivial, i.e. contains no metacharacters, pattern classes, flags, grouping characters etc. (Thanks to Taiyang Li). [#66185](https://github.com/ClickHouse/ClickHouse/pull/66185) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Improvement +* Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). +* The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). +* This PR changes how deduplication for MV works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). +* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). +* In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Make an interactive client for clickhouse-disks, add local disk from the local directory. Fixes [#56791](https://github.com/ClickHouse/ClickHouse/issues/56791). [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). +* Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). +* `StorageS3Queue` related fixes and improvements. Deduce a default value of `s3queue_processing_threads_num` according to the number of physical cpu cores on the server (instead of the previous default value as 1). Set default value of `s3queue_loading_retries` to 10. Fix possible vague "Uncaught exception" in exception column of `system.s3queue`. Do not increment retry count on `MEMORY_LIMIT_EXCEEDED` exception. Move files commit to a stage after insertion into table fully finished to avoid files being commited while not inserted. Add settings `s3queue_max_processed_files_before_commit`, `s3queue_max_processed_rows_before_commit`, `s3queue_max_processed_bytes_before_commit`, `s3queue_max_processing_time_sec_before_commit`, to better control commit and flush time. [#65046](https://github.com/ClickHouse/ClickHouse/pull/65046) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed broken multiple columns aggregation on s390x. [#65062](https://github.com/ClickHouse/ClickHouse/pull/65062) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). +* S3. reduce retires time for queries, increase retries count for backups. 8.5 minutes and 100 retires for queries, 1.2 hours and 1000 retries for backup restore. [#65232](https://github.com/ClickHouse/ClickHouse/pull/65232) ([Sema Checherinda](https://github.com/CheSema)). +* Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). +* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). +* Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). +* Support query plan LIMIT optimization. Support LIMIT pushdown for PostgreSQL storage and table function. [#65454](https://github.com/ClickHouse/ClickHouse/pull/65454) ([Maksim Kita](https://github.com/kitaisreal)). +* Arraymin/max can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). +* Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). +* Do not create format settings for each rows when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). +* Fixed out-of-range exception in parsing Dwarf5 on s390x. [#65501](https://github.com/ClickHouse/ClickHouse/pull/65501) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Reduce `clickhouse-local` prompt to just `:)`. `getFQDNOrHostName()` takes too long on macOS, and we don't want a hostname in the prompt for `clickhouse-local` anyway. [#65510](https://github.com/ClickHouse/ClickHouse/pull/65510) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Avoid printing a message from jemalloc about per-CPU arenas on low-end virtual machines. [#65532](https://github.com/ClickHouse/ClickHouse/pull/65532) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new option to config `` which allow to specify how often clickhouse will reload config. [#65545](https://github.com/ClickHouse/ClickHouse/pull/65545) ([alesapin](https://github.com/alesapin)). +* Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). +* When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection lightweight delete would happen. [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). +* Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). +* `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. This closes [#65695](https://github.com/ClickHouse/ClickHouse/issues/65695). [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a script to backup your files to ClickHouse. This is strange, but works. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* PostgreSQL source support cancel. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). +* Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow to use `concat` function with empty arguments ``` sql :) select concat();. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). +* Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). +* `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). +* Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). +* DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). +* This PR changes how deduplication for MV works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#66144](https://github.com/ClickHouse/ClickHouse/pull/66144) ([Sema Checherinda](https://github.com/CheSema)). +* Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). +* Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). +* Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). +* Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). +* Add settings to ignore ON CLUSTER clause in queries for named collection management with replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Upgraded `pocketfft` dependency to the recent commit https://github.com/mreineck/pocketfft/commit/f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3. [#66291](https://github.com/ClickHouse/ClickHouse/pull/66291) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Upgraded `azure-sdk-for-cpp` to the recent commit https://github.com/ClickHouse/azure-sdk-for-cpp/commit/ea3e19a7be08519134c643177d56c7484dfec884. [#66292](https://github.com/ClickHouse/ClickHouse/pull/66292) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Function `generateSnowflakeID` now allows to specify a machine ID as a parameter to prevent collisions in large clusters. [#66374](https://github.com/ClickHouse/ClickHouse/pull/66374) ([ZAWA_ll](https://github.com/Zawa-ll)). +* Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Closes [#65355](https://github.com/ClickHouse/ClickHouse/issues/65355). Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). +* Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). Fix [#56640](https://github.com/ClickHouse/ClickHouse/issues/56640). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug with cancelation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). +* Fix AST formatting of 'create table b empty as a'. [#64951](https://github.com/ClickHouse/ClickHouse/pull/64951) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Fix unexpected projection name when query with CTE. [#65267](https://github.com/ClickHouse/ClickHouse/pull/65267) ([wudidapaopao](https://github.com/wudidapaopao)). +* Require `dictGet` privilege when accessing dictionaries via direct query or the `Dictionary` table engine. [#65359](https://github.com/ClickHouse/ClickHouse/pull/65359) ([Joe Lynch](https://github.com/joelynch)). +* Fix user-specific S3 auth with incremental backups. [#65481](https://github.com/ClickHouse/ClickHouse/pull/65481) ([Antonio Andelic](https://github.com/antonio2368)). +* Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix getting exception `Index out of bound for blob metadata` in case all files from list batch were filtered out. [#65523](https://github.com/ClickHouse/ClickHouse/pull/65523) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix NOT_FOUND_COLUMN_IN_BLOCK for deduplicate merge of projection. [#65573](https://github.com/ClickHouse/ClickHouse/pull/65573) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed a bug that compatibility level '23.4' was not properly applied. [#65737](https://github.com/ClickHouse/ClickHouse/pull/65737) ([cw5121](https://github.com/cw5121)). +* Fix odbc table with nullable fields. [#65738](https://github.com/ClickHouse/ClickHouse/pull/65738) ([Rodolphe Dugé de Bernonville](https://github.com/RodolpheDuge)). +* Fix data race in `TCPHandler`, which could happen on fatal error. [#65744](https://github.com/ClickHouse/ClickHouse/pull/65744) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Fix a bug leads to EmbeddedRocksDB with TTL write corrupted SST files. [#65816](https://github.com/ClickHouse/ClickHouse/pull/65816) ([Duc Canh Le](https://github.com/canhld94)). +* Functions `bitTest`, `bitTestAll`, and `bitTestAny` now return an error if the specified bit index is out-of-bounds (issue [#65517](https://github.com/ClickHouse/ClickHouse/issues/65517)). [#65818](https://github.com/ClickHouse/ClickHouse/pull/65818) ([Pablo Marcos](https://github.com/pamarcos)). +* Setting `join_any_take_last_row` is supported in any query with hash join. [#65820](https://github.com/ClickHouse/ClickHouse/pull/65820) ([vdimir](https://github.com/vdimir)). +* Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Functions `bitShiftLeft` and `bitShitfRight` return an error for out of bounds shift positions (issue [#65516](https://github.com/ClickHouse/ClickHouse/issues/65516)). [#65838](https://github.com/ClickHouse/ClickHouse/pull/65838) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix growing memory usage in S3Queue. [#65839](https://github.com/ClickHouse/ClickHouse/pull/65839) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix tie handling in `arrayAUC` to match sklearn. [#65840](https://github.com/ClickHouse/ClickHouse/pull/65840) ([gabrielmcg44](https://github.com/gabrielmcg44)). +* Fix possible issues with MySQL server protocol TLS connections. [#65917](https://github.com/ClickHouse/ClickHouse/pull/65917) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible issues with MySQL client protocol TLS connections. [#65938](https://github.com/ClickHouse/ClickHouse/pull/65938) ([Azat Khuzhin](https://github.com/azat)). +* Fix handling of `SSL_ERROR_WANT_READ`/`SSL_ERROR_WANT_WRITE` with zero timeout. [#65941](https://github.com/ClickHouse/ClickHouse/pull/65941) ([Azat Khuzhin](https://github.com/azat)). +* Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Column _size in s3 engine and s3 table function denotes the size of a file inside the archive, not a size of the archive itself. [#65993](https://github.com/ClickHouse/ClickHouse/pull/65993) ([Daniil Ivanik](https://github.com/divanik)). +* Fix resolving dynamic subcolumns in analyzer, avoid reading the whole column on dynamic subcolumn reading. [#66004](https://github.com/ClickHouse/ClickHouse/pull/66004) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix config merging for from_env with replace overrides. [#66034](https://github.com/ClickHouse/ClickHouse/pull/66034) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible hanging in `GRPCServer` during shutdown. This PR fixes [#65622](https://github.com/ClickHouse/ClickHouse/issues/65622). [#66061](https://github.com/ClickHouse/ClickHouse/pull/66061) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` peremeter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixed several cases in function `has` with non-constant `LowCardinality` arguments. [#66088](https://github.com/ClickHouse/ClickHouse/pull/66088) ([Anton Popov](https://github.com/CurtizJ)). +* Fix for `groupArrayIntersect`. It had incorrect behavior in the `merge()` function. Also, fixed behavior in `deserialise()` for numeric and general data. [#66103](https://github.com/ClickHouse/ClickHouse/pull/66103) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the issue when the server failed to parse Avro files with negative block size arrays encoded, which is now allowed by the Avro specification. [#66130](https://github.com/ClickHouse/ClickHouse/pull/66130) ([Serge Klochkov](https://github.com/slvrtrn)). +* Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Fix rare case with missing data in the result of distributed query, close [#61432](https://github.com/ClickHouse/ClickHouse/issues/61432). [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Fix order of parsing metadata fields in StorageDeltaLake. [#66211](https://github.com/ClickHouse/ClickHouse/pull/66211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error reporting while copying to S3 or AzureBlobStorage. [#66295](https://github.com/ClickHouse/ClickHouse/pull/66295) ([Vitaly Baranov](https://github.com/vitlibar)). +* Prevent watchdog from keeping descriptors of unlinked(rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix the bug that logicalexpressionoptimizerpass lost logical type of constant. closes [#64487](https://github.com/ClickHouse/ClickHouse/issues/64487). [#66344](https://github.com/ClickHouse/ClickHouse/pull/66344) ([pn](https://github.com/chloro-pn)). +* Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too aggressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. [#66402](https://github.com/ClickHouse/ClickHouse/pull/66402) ([vdimir](https://github.com/vdimir)). +* Added missing column materialization for cross join. [#66413](https://github.com/ClickHouse/ClickHouse/pull/66413) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid possible logical error during import from Npy format in case of bad array nesting level, fix testing of other kinds of errors. [#66461](https://github.com/ClickHouse/ClickHouse/pull/66461) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix wrong count() result when there is non-deterministic function in predicate. [#66510](https://github.com/ClickHouse/ClickHouse/pull/66510) ([Duc Canh Le](https://github.com/canhld94)). +* Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading of uninitialized memory when hashing empty tuples. This closes [#66559](https://github.com/ClickHouse/ClickHouse/issues/66559). [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix logical error in `PrometheusRequestHandler`. [#66621](https://github.com/ClickHouse/ClickHouse/pull/66621) ([Vitaly Baranov](https://github.com/vitlibar)). +* `column_length` is not updated in `ColumnTuple::insertManyFrom`. [#66626](https://github.com/ClickHouse/ClickHouse/pull/66626) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix rare case of stuck merge after drop column. [#66707](https://github.com/ClickHouse/ClickHouse/pull/66707) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assertion `isUniqTypes` when insert select from remote sources. [#66722](https://github.com/ClickHouse/ClickHouse/pull/66722) ([Sema Checherinda](https://github.com/CheSema)). +* Backported in [#67026](https://github.com/ClickHouse/ClickHouse/issues/67026): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67443](https://github.com/ClickHouse/ClickHouse/issues/67443): Forbid create as select even when database_replicated_allow_heavy_create is set. It was unconditionally forbidden in 23.12 and accidentally allowed under the setting in unreleased 24.7. [#66980](https://github.com/ClickHouse/ClickHouse/pull/66980) ([vdimir](https://github.com/vdimir)). +* Backported in [#67201](https://github.com/ClickHouse/ClickHouse/issues/67201): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67383](https://github.com/ClickHouse/ClickHouse/issues/67383): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67246](https://github.com/ClickHouse/ClickHouse/issues/67246): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Build/Testing/Packaging Improvement +* Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs. [#65186](https://github.com/ClickHouse/ClickHouse/pull/65186) ([Nikita Fomichev](https://github.com/fm4v)). +* Add tests for `base64URLEncode` and `base64URLDecode`. Add analyzer tests. [#65979](https://github.com/ClickHouse/ClickHouse/pull/65979) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix problem when github terminate instances by timeout and artifacts are not collected and full test report is not generated. [#66036](https://github.com/ClickHouse/ClickHouse/pull/66036) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix test [test_grpc_protocol/test.py::test_progress](https://s3.amazonaws.com/clickhouse-test-reports/57695/188f8a3df74caf830ad1ced3c4cf6dfb0aa90093/integration_tests__asan__old_analyzer__[4_6].html). [#66063](https://github.com/ClickHouse/ClickHouse/pull/66063) ([Vitaly Baranov](https://github.com/vitlibar)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs. [#66305](https://github.com/ClickHouse/ClickHouse/pull/66305) ([Nikita Fomichev](https://github.com/fm4v)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs 3. [#66363](https://github.com/ClickHouse/ClickHouse/pull/66363) ([Nikita Fomichev](https://github.com/fm4v)). +* Tests: fix tests hang up in cases when gdb catches error. [#66411](https://github.com/ClickHouse/ClickHouse/pull/66411) ([Nikita Fomichev](https://github.com/fm4v)). +* ... since [Release v24.6.1.4423-stable](https://github.com/ClickHouse/ClickHouse/releases/tag/v24.6.1.4423-stable) when build in ppc64le with dynamic openssl build (`cmake -DENABLE_OPENSSL_DYNAMIC=1 -DCMAKE_TOOLCHAIN_FILE= cmake/linux/toolchain-ppc64le.cmake `) got error: ` ld.lld: error: duplicate symbol: OPENSSL_cleanse`. [#66733](https://github.com/ClickHouse/ClickHouse/pull/66733) ([Yong Wang](https://github.com/kashwy)). + +#### NO CL CATEGORY + +* Backported in [#67084](https://github.com/ClickHouse/ClickHouse/issues/67084):. [#67040](https://github.com/ClickHouse/ClickHouse/pull/67040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67452](https://github.com/ClickHouse/ClickHouse/issues/67452):. [#67392](https://github.com/ClickHouse/ClickHouse/pull/67392) ([alesapin](https://github.com/alesapin)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Revert "Small fix for 02340_parts_refcnt_mergetree""'. [#65155](https://github.com/ClickHouse/ClickHouse/pull/65155) ([Nikita Taranov](https://github.com/nickitat)). +* NO CL ENTRY: 'Revert "Use 1MB HTTP buffers to avoid frequent send syscalls"'. [#65498](https://github.com/ClickHouse/ClickHouse/pull/65498) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Resubmit http_external_tables_memory_tracking test"'. [#65500](https://github.com/ClickHouse/ClickHouse/pull/65500) ([Nikita Taranov](https://github.com/nickitat)). +* NO CL ENTRY: 'Revert "Add an assertion in ReplicatedMergeTreeQueue"'. [#65686](https://github.com/ClickHouse/ClickHouse/pull/65686) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "insertion deduplication on retries for materialised views"'. [#66134](https://github.com/ClickHouse/ClickHouse/pull/66134) ([Sema Checherinda](https://github.com/CheSema)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Return and fix 01600_parts_states_metrics_long test. [#58748](https://github.com/ClickHouse/ClickHouse/pull/58748) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add azure_cache as storage policy to tests. [#59943](https://github.com/ClickHouse/ClickHouse/pull/59943) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Minor: replaced expression with LEGACY_MAX_LEVEL. [#61268](https://github.com/ClickHouse/ClickHouse/pull/61268) ([Vasily Nemkov](https://github.com/Enmk)). +* Make write to temporary data in cache do all checks and assertions as during write to ordinary cache. [#63348](https://github.com/ClickHouse/ClickHouse/pull/63348) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Refactoring near azure blob storage. [#63636](https://github.com/ClickHouse/ClickHouse/pull/63636) ([Anton Popov](https://github.com/CurtizJ)). +* Everything should work with Analyzer. [#63643](https://github.com/ClickHouse/ClickHouse/pull/63643) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* add some log for non using detached tables. [#64992](https://github.com/ClickHouse/ClickHouse/pull/64992) ([Konstantin Morozov](https://github.com/k-morozov)). +* Remove dag flags. [#65234](https://github.com/ClickHouse/ClickHouse/pull/65234) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix flaky autocompletion test. [#65246](https://github.com/ClickHouse/ClickHouse/pull/65246) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Disable userspace page cache by default. [#65305](https://github.com/ClickHouse/ClickHouse/pull/65305) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v24.4.3.25-stable. [#65308](https://github.com/ClickHouse/ClickHouse/pull/65308) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Do not raise a NOT_IMPLEMENTED error when getting s3 metrics with a multiple disk configuration. [#65403](https://github.com/ClickHouse/ClickHouse/pull/65403) ([Elena Torró](https://github.com/elenatorro)). +* Dodging reading from wrong table with parallel replicas. [#65417](https://github.com/ClickHouse/ClickHouse/pull/65417) ([Nikita Taranov](https://github.com/nickitat)). +* Fix: return error if can't connect to any replicas chosen for query execution. [#65467](https://github.com/ClickHouse/ClickHouse/pull/65467) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `AzureObjectStorage::exists` method. [#65471](https://github.com/ClickHouse/ClickHouse/pull/65471) ([Anton Popov](https://github.com/CurtizJ)). +* Update version after release. [#65483](https://github.com/ClickHouse/ClickHouse/pull/65483) ([Raúl Marín](https://github.com/Algunenano)). +* Generate 24.6 changelog. [#65485](https://github.com/ClickHouse/ClickHouse/pull/65485) ([Raúl Marín](https://github.com/Algunenano)). +* Fix of `PlanSquashingTransform`: pipeline stuck. [#65487](https://github.com/ClickHouse/ClickHouse/pull/65487) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix bad test `02922_deduplication_with_zero_copy`. [#65492](https://github.com/ClickHouse/ClickHouse/pull/65492) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable clang-format in special areas. [#65495](https://github.com/ClickHouse/ClickHouse/pull/65495) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `test_keeper_snapshots`. [#65497](https://github.com/ClickHouse/ClickHouse/pull/65497) ([Antonio Andelic](https://github.com/antonio2368)). +* Update to libunwind 8.1.7. [#65509](https://github.com/ClickHouse/ClickHouse/pull/65509) ([Michael Kolupaev](https://github.com/al13n321)). +* Setting `uniform_snowflake_conversion_functions` (not in any release yet) was replaced by setting `allow_deprecated_snowflake_conversion_functions`. The latter controls if the legacy snowflake conversion functions are available (by default, they are not). [#65522](https://github.com/ClickHouse/ClickHouse/pull/65522) ([Robert Schulze](https://github.com/rschu1ze)). +* Try CI without RerunCheck, jobs can be easily rerun manually though extra amount of work in CI will follow on workflow restart. [#65524](https://github.com/ClickHouse/ClickHouse/pull/65524) ([Max K.](https://github.com/maxknv)). +* Bump re2 to latest HEAD. [#65526](https://github.com/ClickHouse/ClickHouse/pull/65526) ([Robert Schulze](https://github.com/rschu1ze)). +* OpenSSL: Replace temporary fix for unsynchronized access by official fix. [#65529](https://github.com/ClickHouse/ClickHouse/pull/65529) ([Robert Schulze](https://github.com/rschu1ze)). +* Update README.md. [#65531](https://github.com/ClickHouse/ClickHouse/pull/65531) ([Tyler Hannan](https://github.com/tylerhannan)). +* CI: some time there are timeouts on DROP TABLES for random tests. [#65535](https://github.com/ClickHouse/ClickHouse/pull/65535) ([Sema Checherinda](https://github.com/CheSema)). +* Synchronize `MARK_CACHE_SIZE` value in default settings and config. [#65547](https://github.com/ClickHouse/ClickHouse/pull/65547) ([Denny Crane](https://github.com/den-crane)). +* CI: Skip removed test files in stateless flaky check job. [#65553](https://github.com/ClickHouse/ClickHouse/pull/65553) ([Max K.](https://github.com/maxknv)). +* Renames Build report jobs. [#65554](https://github.com/ClickHouse/ClickHouse/pull/65554) ([Max K.](https://github.com/maxknv)). +* Parse user from URL for dashboard.html (useful for sharing). [#65556](https://github.com/ClickHouse/ClickHouse/pull/65556) ([Azat Khuzhin](https://github.com/azat)). +* Remove tech debt. [#65561](https://github.com/ClickHouse/ClickHouse/pull/65561) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Maybe fix test `00763_lock_buffer_long.sh`. [#65562](https://github.com/ClickHouse/ClickHouse/pull/65562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix clickhouse-keeper with not system-wide directories and provide override for local development. [#65563](https://github.com/ClickHouse/ClickHouse/pull/65563) ([Azat Khuzhin](https://github.com/azat)). +* Re-configure yamllint to allow document-start. [#65565](https://github.com/ClickHouse/ClickHouse/pull/65565) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky test `01254_dict_load_after_detach_attach.sql`. [#65571](https://github.com/ClickHouse/ClickHouse/pull/65571) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve flaky test to provide more diagnostics. [#65586](https://github.com/ClickHouse/ClickHouse/pull/65586) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_parallel_replicas_distributed_skip_shards flakiness. [#65588](https://github.com/ClickHouse/ClickHouse/pull/65588) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix an error in the test about metadata_type. [#65592](https://github.com/ClickHouse/ClickHouse/pull/65592) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix slow test. [#65593](https://github.com/ClickHouse/ClickHouse/pull/65593) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fix flaky 02864_statistics_uniq. [#65599](https://github.com/ClickHouse/ClickHouse/pull/65599) ([Han Fei](https://github.com/hanfei1991)). +* Fix 03172_error_log_table_not_empty. [#65604](https://github.com/ClickHouse/ClickHouse/pull/65604) ([Pablo Marcos](https://github.com/pamarcos)). +* Enable realtime digest for Jepsen tests. [#65608](https://github.com/ClickHouse/ClickHouse/pull/65608) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: Return Job Rerun check. [#65613](https://github.com/ClickHouse/ClickHouse/pull/65613) ([Max K.](https://github.com/maxknv)). +* Update CHANGELOG.md. [#65624](https://github.com/ClickHouse/ClickHouse/pull/65624) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Module is required for authenticating in GH (in cloud). [#65628](https://github.com/ClickHouse/ClickHouse/pull/65628) ([Max K.](https://github.com/maxknv)). +* Update IObjectStorage.h. [#65631](https://github.com/ClickHouse/ClickHouse/pull/65631) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix 02834_apache_arrow_abort flakiness with MSAN. [#65640](https://github.com/ClickHouse/ClickHouse/pull/65640) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix overflow in StorageWindowView. [#65641](https://github.com/ClickHouse/ClickHouse/pull/65641) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix inconsistent AST formatting when a keyword is used as type name. [#65648](https://github.com/ClickHouse/ClickHouse/pull/65648) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Single point of setting mergeable check status. [#65658](https://github.com/ClickHouse/ClickHouse/pull/65658) ([Max K.](https://github.com/maxknv)). +* Miscellaneous and insignificant changes around Client/ClientBase. [#65669](https://github.com/ClickHouse/ClickHouse/pull/65669) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add Replicated database names to ZooKeeper for introspection. [#65675](https://github.com/ClickHouse/ClickHouse/pull/65675) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Introduce type contract checks in `IColumn`. [#65687](https://github.com/ClickHouse/ClickHouse/pull/65687) ([Nikita Taranov](https://github.com/nickitat)). +* Print slightly more information in 02982_aggregation_states_destruction. [#65688](https://github.com/ClickHouse/ClickHouse/pull/65688) ([Michael Kolupaev](https://github.com/al13n321)). +* Disable stacktrace collection in GWPAsan by default. [#65701](https://github.com/ClickHouse/ClickHouse/pull/65701) ([Antonio Andelic](https://github.com/antonio2368)). +* Build jemalloc with profiler. [#65702](https://github.com/ClickHouse/ClickHouse/pull/65702) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix duplicate symbol linkage error. [#65705](https://github.com/ClickHouse/ClickHouse/pull/65705) ([Nikita Taranov](https://github.com/nickitat)). +* Fix server restarts in performance tests. [#65717](https://github.com/ClickHouse/ClickHouse/pull/65717) ([Antonio Andelic](https://github.com/antonio2368)). +* Update 03002_part_log_rmt_fetch_mutate_error.sql. [#65720](https://github.com/ClickHouse/ClickHouse/pull/65720) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky `02265_column_ttl`. Closes [#65719](https://github.com/ClickHouse/ClickHouse/issues/65719). [#65742](https://github.com/ClickHouse/ClickHouse/pull/65742) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* See [#65745](https://github.com/ClickHouse/ClickHouse/issues/65745). It doesn't solve the issue, but helps a bit. [#65746](https://github.com/ClickHouse/ClickHouse/pull/65746) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update CHANGELOG.md. [#65752](https://github.com/ClickHouse/ClickHouse/pull/65752) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* document declarative ssh-keys authentication. [#65756](https://github.com/ClickHouse/ClickHouse/pull/65756) ([Tobias Florek](https://github.com/ibotty)). +* `base64En/Decode64Url` --> `base64En/Decode64URL`. [#65760](https://github.com/ClickHouse/ClickHouse/pull/65760) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix for issue [#65757](https://github.com/ClickHouse/ClickHouse/issues/65757). [#65763](https://github.com/ClickHouse/ClickHouse/pull/65763) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix flaky `test_replicated_database::test_alter_attach`. [#65766](https://github.com/ClickHouse/ClickHouse/pull/65766) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix: progress bar for read in order queries. [#65769](https://github.com/ClickHouse/ClickHouse/pull/65769) ([Igor Nikonov](https://github.com/devcrafter)). +* CI: Fix for Builds report job in backports and releases. [#65774](https://github.com/ClickHouse/ClickHouse/pull/65774) ([Max K.](https://github.com/maxknv)). +* CI: New create release workflow. [#65775](https://github.com/ClickHouse/ClickHouse/pull/65775) ([Max K.](https://github.com/maxknv)). +* fixed misspelled word. [#65778](https://github.com/ClickHouse/ClickHouse/pull/65778) ([Linh Giang](https://github.com/linhgiang24)). +* Refactor statistics interface. [#65792](https://github.com/ClickHouse/ClickHouse/pull/65792) ([Robert Schulze](https://github.com/rschu1ze)). +* Try to make `test_ldap_external_user_directory` less flaky. [#65794](https://github.com/ClickHouse/ClickHouse/pull/65794) ([Andrey Zvonov](https://github.com/zvonand)). +* AMI image with gh and jwt. [#65795](https://github.com/ClickHouse/ClickHouse/pull/65795) ([Max K.](https://github.com/maxknv)). +* Forbid join algorithm randomisation for 03094_one_thousand_joins. [#65798](https://github.com/ClickHouse/ClickHouse/pull/65798) ([Nikita Taranov](https://github.com/nickitat)). +* Fix 02931_rewrite_sum_column_and_constant flakiness. [#65800](https://github.com/ClickHouse/ClickHouse/pull/65800) ([Michael Kolupaev](https://github.com/al13n321)). +* Update StorageMaterializedView.cpp. [#65801](https://github.com/ClickHouse/ClickHouse/pull/65801) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix slow `getFQDNOrHostNameImpl` on macOS. [#65803](https://github.com/ClickHouse/ClickHouse/pull/65803) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* No jemalloc profiler for non-Linux. [#65834](https://github.com/ClickHouse/ClickHouse/pull/65834) ([Antonio Andelic](https://github.com/antonio2368)). +* Add missing workload identity changes. [#65848](https://github.com/ClickHouse/ClickHouse/pull/65848) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix rocksdb. [#65858](https://github.com/ClickHouse/ClickHouse/pull/65858) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update the list of easy tasks. [#65865](https://github.com/ClickHouse/ClickHouse/pull/65865) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#65866](https://github.com/ClickHouse/ClickHouse/pull/65866) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#43003](https://github.com/ClickHouse/ClickHouse/issues/43003). [#65870](https://github.com/ClickHouse/ClickHouse/pull/65870) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Uninteresting changes. [#65871](https://github.com/ClickHouse/ClickHouse/pull/65871) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Max sessions for user tests improvements. [#65888](https://github.com/ClickHouse/ClickHouse/pull/65888) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Update version_date.tsv and changelogs after v24.6.1.4423-stable. [#65909](https://github.com/ClickHouse/ClickHouse/pull/65909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Remove standalone Keeper build. [#65910](https://github.com/ClickHouse/ClickHouse/pull/65910) ([Antonio Andelic](https://github.com/antonio2368)). +* Add extra profiling helpers for Keeper. [#65918](https://github.com/ClickHouse/ClickHouse/pull/65918) ([Antonio Andelic](https://github.com/antonio2368)). +* PostgreSQL source cancel query comments. [#65919](https://github.com/ClickHouse/ClickHouse/pull/65919) ([Maksim Kita](https://github.com/kitaisreal)). +* Remove mysqlxx::Pool::Entry assignment operator. [#65920](https://github.com/ClickHouse/ClickHouse/pull/65920) ([Azat Khuzhin](https://github.com/azat)). +* No random settings for a test with `Object(JSON)`. [#65921](https://github.com/ClickHouse/ClickHouse/pull/65921) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Follow up to [#65046](https://github.com/ClickHouse/ClickHouse/issues/65046). [#65928](https://github.com/ClickHouse/ClickHouse/pull/65928) ([Kseniia Sumarokova](https://github.com/kssenii)). +* add restriction for storage join. [#65936](https://github.com/ClickHouse/ClickHouse/pull/65936) ([Han Fei](https://github.com/hanfei1991)). +* Update version_date.tsv and changelogs after v24.5.4.49-stable. [#65937](https://github.com/ClickHouse/ClickHouse/pull/65937) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Add table name to MergeTreeSource spans. [#65940](https://github.com/ClickHouse/ClickHouse/pull/65940) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SettingsChangesHistory 24.7. [#65945](https://github.com/ClickHouse/ClickHouse/pull/65945) ([Raúl Marín](https://github.com/Algunenano)). +* Fix logical error "Expected ReadBufferFromFile, but got DB::EmptyReadBuffer". [#65949](https://github.com/ClickHouse/ClickHouse/pull/65949) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use -Og instead of -O0 for debug builds. [#65953](https://github.com/ClickHouse/ClickHouse/pull/65953) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix data race for Keeper snapshot queue. [#65970](https://github.com/ClickHouse/ClickHouse/pull/65970) ([Antonio Andelic](https://github.com/antonio2368)). +* Minor changes in CHANGELOG. [#65971](https://github.com/ClickHouse/ClickHouse/pull/65971) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove unnatural punctuation from Parquet. [#65972](https://github.com/ClickHouse/ClickHouse/pull/65972) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try fix "Check timeout expired" without any server logs in report in stateless tests. [#65977](https://github.com/ClickHouse/ClickHouse/pull/65977) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix support of non-const scale arguments in rounding functions. [#65983](https://github.com/ClickHouse/ClickHouse/pull/65983) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* More aesthetic error messages. [#65985](https://github.com/ClickHouse/ClickHouse/pull/65985) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix race in s3queue. [#65986](https://github.com/ClickHouse/ClickHouse/pull/65986) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Now it's possible to specify `s3-storage`, `azure-object-storage` and in general `object-storage`. [#65988](https://github.com/ClickHouse/ClickHouse/pull/65988) ([alesapin](https://github.com/alesapin)). +* Fix flaky test_storage_s3_queue tests. [#66009](https://github.com/ClickHouse/ClickHouse/pull/66009) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Relax the check in 02982_aggregation_states_destruction. [#66011](https://github.com/ClickHouse/ClickHouse/pull/66011) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `01158_zookeeper_log_long`. [#66012](https://github.com/ClickHouse/ClickHouse/pull/66012) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove scary jemalloc log. [#66028](https://github.com/ClickHouse/ClickHouse/pull/66028) ([Antonio Andelic](https://github.com/antonio2368)). +* Move experimental settings to the experimental block. [#66030](https://github.com/ClickHouse/ClickHouse/pull/66030) ([Raúl Marín](https://github.com/Algunenano)). +* Fix lock-order-inversion in DatabaseCatalog. [#66038](https://github.com/ClickHouse/ClickHouse/pull/66038) ([Nikolay Degterinsky](https://github.com/evillique)). +* Try disabling jemalloc background threads. [#66041](https://github.com/ClickHouse/ClickHouse/pull/66041) ([Antonio Andelic](https://github.com/antonio2368)). +* Try to avoid conflicts in `SettingsChangesHistory.cpp`. [#66042](https://github.com/ClickHouse/ClickHouse/pull/66042) ([Anton Popov](https://github.com/CurtizJ)). +* Add profile events for regex cache. [#66050](https://github.com/ClickHouse/ClickHouse/pull/66050) ([Antonio Andelic](https://github.com/antonio2368)). +* Bump vectorscan to 5.4.10.1. [#66056](https://github.com/ClickHouse/ClickHouse/pull/66056) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove obsolete comment. [#66059](https://github.com/ClickHouse/ClickHouse/pull/66059) ([Robert Schulze](https://github.com/rschu1ze)). +* Maybe fix tsan assert in `test_mysql_killed_while_insert_8_0`. [#66064](https://github.com/ClickHouse/ClickHouse/pull/66064) ([Robert Schulze](https://github.com/rschu1ze)). +* Move some of `HTTPHandler` stuff to separate source files in order to reuse it in `PrometheusRequestHandler`. This PR is required for https://github.com/ClickHouse/ClickHouse/pull/64183. [#66067](https://github.com/ClickHouse/ClickHouse/pull/66067) ([Vitaly Baranov](https://github.com/vitlibar)). +* Bump rocksdb to v6.23.3. [#66068](https://github.com/ClickHouse/ClickHouse/pull/66068) ([Robert Schulze](https://github.com/rschu1ze)). +* Add protobufs for `Prometheus` `remote-write` / `remote-read` protocols to our repository. Fix cmake script for compiling protobufs. [#66069](https://github.com/ClickHouse/ClickHouse/pull/66069) ([Vitaly Baranov](https://github.com/vitlibar)). +* Use pinned versions of all python packages in CI docker images. Also makes clang-18.1.8 work with sanitizers and surprisingly fixes [#66049](https://github.com/ClickHouse/ClickHouse/issues/66049). [#66070](https://github.com/ClickHouse/ClickHouse/pull/66070) ([alesapin](https://github.com/alesapin)). +* Clean-up custom LLVM 15 patches. [#66072](https://github.com/ClickHouse/ClickHouse/pull/66072) ([Robert Schulze](https://github.com/rschu1ze)). +* Minor JWT client fixes. [#66073](https://github.com/ClickHouse/ClickHouse/pull/66073) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Bump vectorscan to 5.4.11. [#66082](https://github.com/ClickHouse/ClickHouse/pull/66082) ([Robert Schulze](https://github.com/rschu1ze)). +* Print stacktrace in case of abort after logical error. [#66091](https://github.com/ClickHouse/ClickHouse/pull/66091) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix flaky 03172_error_log_table_not_empty. [#66093](https://github.com/ClickHouse/ClickHouse/pull/66093) ([Sema Checherinda](https://github.com/CheSema)). +* Bump s2geometry to latest master. [#66094](https://github.com/ClickHouse/ClickHouse/pull/66094) ([Robert Schulze](https://github.com/rschu1ze)). +* update keeper bench example config file. [#66095](https://github.com/ClickHouse/ClickHouse/pull/66095) ([Han Fei](https://github.com/hanfei1991)). +* Avoid using source directory for generated files. [#66097](https://github.com/ClickHouse/ClickHouse/pull/66097) ([Azat Khuzhin](https://github.com/azat)). +* More precise warning message about sanitizers. [#66098](https://github.com/ClickHouse/ClickHouse/pull/66098) ([Anton Popov](https://github.com/CurtizJ)). +* Slightly better calculation of primary index. [#66099](https://github.com/ClickHouse/ClickHouse/pull/66099) ([Anton Popov](https://github.com/CurtizJ)). +* Bump Azure to 1.12. [#66100](https://github.com/ClickHouse/ClickHouse/pull/66100) ([Robert Schulze](https://github.com/rschu1ze)). +* Add a test for [#58998](https://github.com/ClickHouse/ClickHouse/issues/58998). [#66101](https://github.com/ClickHouse/ClickHouse/pull/66101) ([Anton Popov](https://github.com/CurtizJ)). +* CI: Fix sync pr merge. [#66105](https://github.com/ClickHouse/ClickHouse/pull/66105) ([Max K.](https://github.com/maxknv)). +* Remove flaky case from 02956_rocksdb_bulk_sink. [#66107](https://github.com/ClickHouse/ClickHouse/pull/66107) ([vdimir](https://github.com/vdimir)). +* Fix bugfix checker. [#66120](https://github.com/ClickHouse/ClickHouse/pull/66120) ([Raúl Marín](https://github.com/Algunenano)). +* Correctly print long processing requests in Keeper. [#66124](https://github.com/ClickHouse/ClickHouse/pull/66124) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v24.6.2.17-stable. [#66127](https://github.com/ClickHouse/ClickHouse/pull/66127) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Bump s2geometry again. [#66136](https://github.com/ClickHouse/ClickHouse/pull/66136) ([Robert Schulze](https://github.com/rschu1ze)). +* Switch submodule `contrib/orc` to a proper commit in the [main](https://github.com/ClickHouse/orc/tree/main) branch. Previously a commit from a removed branch was used ([see](https://github.com/ClickHouse/orc/pull/13)). [#66137](https://github.com/ClickHouse/ClickHouse/pull/66137) ([Vitaly Baranov](https://github.com/vitlibar)). +* Finalize MergedBlockOutputStream in dtor. [#66138](https://github.com/ClickHouse/ClickHouse/pull/66138) ([Nikita Taranov](https://github.com/nickitat)). +* Proper destruction order of AsyncLoader::Pool fields. [#66145](https://github.com/ClickHouse/ClickHouse/pull/66145) ([Sergei Trifonov](https://github.com/serxa)). +* Playing minesweeper with build system. [#66147](https://github.com/ClickHouse/ClickHouse/pull/66147) ([Nikita Taranov](https://github.com/nickitat)). +* Fix clang-tidy error in BufferWithOwnMemory.h. [#66161](https://github.com/ClickHouse/ClickHouse/pull/66161) ([Nikita Taranov](https://github.com/nickitat)). +* Use peak_threads_usage instead of arrayUniq(thread_ids) in tests. [#66162](https://github.com/ClickHouse/ClickHouse/pull/66162) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash when adding empty tuple to query cache. [#66168](https://github.com/ClickHouse/ClickHouse/pull/66168) ([Michael Kolupaev](https://github.com/al13n321)). +* tests: fix 01563_distributed_query_finish flakiness (due to system.*_log_sender). [#66171](https://github.com/ClickHouse/ClickHouse/pull/66171) ([Azat Khuzhin](https://github.com/azat)). +* Refactor `OptimizeIfWithConstantConditionVisitor` using `InDepthNodeVisitor`. [#66184](https://github.com/ClickHouse/ClickHouse/pull/66184) ([zhongyuankai](https://github.com/zhongyuankai)). +* Update README.md. [#66186](https://github.com/ClickHouse/ClickHouse/pull/66186) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix 01246_buffer_flush flakiness. [#66188](https://github.com/ClickHouse/ClickHouse/pull/66188) ([Azat Khuzhin](https://github.com/azat)). +* Avoid using harmful function `rand()` in grpc. [#66191](https://github.com/ClickHouse/ClickHouse/pull/66191) ([Vitaly Baranov](https://github.com/vitlibar)). +* Bump RocksDB. [#66216](https://github.com/ClickHouse/ClickHouse/pull/66216) ([Robert Schulze](https://github.com/rschu1ze)). +* Update README.md. [#66217](https://github.com/ClickHouse/ClickHouse/pull/66217) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fixes peak_threads_usage metric when materialised views are involved. [#66230](https://github.com/ClickHouse/ClickHouse/pull/66230) ([Sema Checherinda](https://github.com/CheSema)). +* Remove test as requested in https://github.com/ClickHouse/ClickHouse/pull/65277#issuecomment-2211361465. [#66233](https://github.com/ClickHouse/ClickHouse/pull/66233) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix test `00504_mergetree_arrays_rw.sql`. [#66248](https://github.com/ClickHouse/ClickHouse/pull/66248) ([Anton Popov](https://github.com/CurtizJ)). +* CI: Do not finalize CI running status unless all success. [#66276](https://github.com/ClickHouse/ClickHouse/pull/66276) ([Max K.](https://github.com/maxknv)). +* Collect core dumps in more tests. [#66281](https://github.com/ClickHouse/ClickHouse/pull/66281) ([Antonio Andelic](https://github.com/antonio2368)). +* Add a stateless test for gRPC protocol. [#66284](https://github.com/ClickHouse/ClickHouse/pull/66284) ([Vitaly Baranov](https://github.com/vitlibar)). +* Log message: Failed to connect to replica ... [#66289](https://github.com/ClickHouse/ClickHouse/pull/66289) ([Igor Nikonov](https://github.com/devcrafter)). +* Update run.sh. [#66290](https://github.com/ClickHouse/ClickHouse/pull/66290) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Some changes in the codebase as a preparation for LLVM 18. [#66293](https://github.com/ClickHouse/ClickHouse/pull/66293) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* There's some problem with iptables in parallel tests. [#66304](https://github.com/ClickHouse/ClickHouse/pull/66304) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Build failure if ENABLE_AWS_S3 is OFF fixed. [#66335](https://github.com/ClickHouse/ClickHouse/pull/66335) ([Ilya Golshtein](https://github.com/ilejn)). +* Enable checks in assert_cast under sanitizers. [#66336](https://github.com/ClickHouse/ClickHouse/pull/66336) ([Nikita Taranov](https://github.com/nickitat)). +* Create release workflow. [#66339](https://github.com/ClickHouse/ClickHouse/pull/66339) ([Max K.](https://github.com/maxknv)). +* Fix invalid XML. [#66342](https://github.com/ClickHouse/ClickHouse/pull/66342) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix possible deadlock for jemalloc with enabled profiler. [#66346](https://github.com/ClickHouse/ClickHouse/pull/66346) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_parallel_replicas_custom_key. [#66349](https://github.com/ClickHouse/ClickHouse/pull/66349) ([Antonio Andelic](https://github.com/antonio2368)). +* Collect logs from `minio` in stateless and statefull tests. [#66353](https://github.com/ClickHouse/ClickHouse/pull/66353) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix "Sending a batch of X files to Y (0.00 rows, 0.00 B bytes)." in case of batch restoring. [#66375](https://github.com/ClickHouse/ClickHouse/pull/66375) ([Azat Khuzhin](https://github.com/azat)). +* Fix 03030_system_flush_distributed_settings flakiness. [#66376](https://github.com/ClickHouse/ClickHouse/pull/66376) ([Azat Khuzhin](https://github.com/azat)). +* PR cleanup: remove redundant code. [#66380](https://github.com/ClickHouse/ClickHouse/pull/66380) ([Igor Nikonov](https://github.com/devcrafter)). +* New slack bot to post messages about CI events - Post message if OOM. [#66392](https://github.com/ClickHouse/ClickHouse/pull/66392) ([Max K.](https://github.com/maxknv)). +* Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_rabbitmq/test.py. [#66396](https://github.com/ClickHouse/ClickHouse/pull/66396) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add query elapsed time for non-default format in play UI. [#66398](https://github.com/ClickHouse/ClickHouse/pull/66398) ([Azat Khuzhin](https://github.com/azat)). +* Untangle setting headers. [#66404](https://github.com/ClickHouse/ClickHouse/pull/66404) ([Raúl Marín](https://github.com/Algunenano)). +* Remove noisy message. [#66406](https://github.com/ClickHouse/ClickHouse/pull/66406) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If job detected as in-progress in PR workflow run - just remove it from todo list, as it not affected by the change. [#66407](https://github.com/ClickHouse/ClickHouse/pull/66407) ([Max K.](https://github.com/maxknv)). +* CI: CIBuddy to post from master only. [#66417](https://github.com/ClickHouse/ClickHouse/pull/66417) ([Max K.](https://github.com/maxknv)). +* Add a test for [#66333](https://github.com/ClickHouse/ClickHouse/issues/66333). [#66432](https://github.com/ClickHouse/ClickHouse/pull/66432) ([max-vostrikov](https://github.com/max-vostrikov)). +* Limit number of linker jobs on arm to avoid OOM during build. [#66435](https://github.com/ClickHouse/ClickHouse/pull/66435) ([Nikita Taranov](https://github.com/nickitat)). +* [RFC] Fix jemalloc assertion due to non-monotonic CLOCK_MONOTONIC_COARSE. [#66439](https://github.com/ClickHouse/ClickHouse/pull/66439) ([Azat Khuzhin](https://github.com/azat)). +* CI: Do not block CI on few number of test failures. [#66440](https://github.com/ClickHouse/ClickHouse/pull/66440) ([Max K.](https://github.com/maxknv)). +* Stateless tests: fix flaky tests 01037_polygon_dicts*. [#66445](https://github.com/ClickHouse/ClickHouse/pull/66445) ([Nikita Fomichev](https://github.com/fm4v)). +* Related to https://github.com/ClickHouse/ClickHouse/pull/62067 https://s3.amazonaws.com/clickhouse-test-reports/66410/5557dce188cabc7477bb4e874d47e3b80278ee66/stateless_tests__release_.html ``` 2024-07-12 16:04:29 +Queries for alter_table did not finish automatically after 250+ seconds 2024-07-12 16:04:29 +==================== QUERIES ==================== 2024-07-12 16:04:29 +Row 1: 2024-07-12 16:04:29 +────── 2024-07-12 16:04:29 +is_initial_query: 1 2024-07-12 16:04:29 +user: default 2024-07-12 16:04:29 +query_id: b43ffd7d-aee6-4161-aa82-bf9fff9d78c0 2024-07-12 16:04:29 +address: ::1 2024-07-12 16:04:29 +port: 58360 2024-07-12 16:04:29 +initial_user: default 2024-07-12 16:04:29 +initial_query_id: b43ffd7d-aee6-4161-aa82-bf9fff9d78c0 ... 2024-07-12 16:04:29 +query: OPTIMIZE TABLE alter_table0 FINAL ```. [#66460](https://github.com/ClickHouse/ClickHouse/pull/66460) ([Alexander Tokmakov](https://github.com/tavplubix)). +* OOM error was not visible since process is killed and status is not set Change sets ERROR status if job was killed. [#66463](https://github.com/ClickHouse/ClickHouse/pull/66463) ([Max K.](https://github.com/maxknv)). +* Add AST fuzzers jobs for CI caching so that they can be skipped in PRs not related to build or tests. [#66468](https://github.com/ClickHouse/ClickHouse/pull/66468) ([Max K.](https://github.com/maxknv)). +* If job with the same digest has been seen in master's CI it should be skipped in PR run. [#66471](https://github.com/ClickHouse/ClickHouse/pull/66471) ([Max K.](https://github.com/maxknv)). +* CI: Check job's exit status and report if killed. [#66477](https://github.com/ClickHouse/ClickHouse/pull/66477) ([Max K.](https://github.com/maxknv)). +* This closes [#37557](https://github.com/ClickHouse/ClickHouse/issues/37557). [#66482](https://github.com/ClickHouse/ClickHouse/pull/66482) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Add retry for GH set_status_comment call. [#66488](https://github.com/ClickHouse/ClickHouse/pull/66488) ([Max K.](https://github.com/maxknv)). +* OpenSSL: Minor follow-up to [#66064](https://github.com/ClickHouse/ClickHouse/issues/66064). [#66489](https://github.com/ClickHouse/ClickHouse/pull/66489) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Fix for job filtering in PRs. [#66490](https://github.com/ClickHouse/ClickHouse/pull/66490) ([Max K.](https://github.com/maxknv)). +* CI: Create release workflow updates. [#66498](https://github.com/ClickHouse/ClickHouse/pull/66498) ([Max K.](https://github.com/maxknv)). +* Add one more revision to ignore. [#66499](https://github.com/ClickHouse/ClickHouse/pull/66499) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unit tests vomit a ton of garbage, see https://s3.amazonaws.com/clickhouse-test-reports/66457/0c82dc91f07b29ba503d7579c7d3ebecba532b73/unit_tests__tsan_/run.log - remove it. [#66501](https://github.com/ClickHouse/ClickHouse/pull/66501) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix double whitespace in EXPLAIN AST CREATE. [#66505](https://github.com/ClickHouse/ClickHouse/pull/66505) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02530_dictionaries_update_field`. [#66507](https://github.com/ClickHouse/ClickHouse/pull/66507) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Https://play.clickhouse.com/play?user=play#u0vmrunuignozwnrx3n0yxj0x3rpbwusignozwnrx25hbwusihrlc3rfbmftzswgcmvwb3j0x3vybapguk9nignozwnrcwpxsevsrsbjagvja19zdgfydf90aw1lid49ig5vdygpic0gsu5urvjwquwgmjqwiehpvvikicagieforcbwdwxsx3jlcxvlc3rfbnvtymvyid0gmaogicagqu5eihrlc3rfc3rhdhvzice9icdts0lquevejwogicagqu5eihrlc3rfc3rhdhvziexjs0ugj0yljwogicagqu5eignozwnrx3n0yxr1cyahpsanc3vjy2vzcyckicagieforcbwb3npdglvbih0zxn0x25hbwusicdhcgfjagvfyxjyb3cnksa+idakt1jervigqlkgy2hly2tfc3rhcnrfdgltzq==. [#66508](https://github.com/ClickHouse/ClickHouse/pull/66508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MSan report in GRPC. [#66509](https://github.com/ClickHouse/ClickHouse/pull/66509) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Fix for skipping Builds_2 in PRs' CI. [#66512](https://github.com/ClickHouse/ClickHouse/pull/66512) ([Max K.](https://github.com/maxknv)). +* CI: Do not block Tests_3 unless MAX_FAILED_TESTS exceeded. [#66513](https://github.com/ClickHouse/ClickHouse/pull/66513) ([Max K.](https://github.com/maxknv)). +* Fix `02918_parallel_replicas_custom_key_unavailable_replica`. [#66516](https://github.com/ClickHouse/ClickHouse/pull/66516) ([Antonio Andelic](https://github.com/antonio2368)). +* Stateless tests: improvements related to OOM of test runs. [#66520](https://github.com/ClickHouse/ClickHouse/pull/66520) ([Nikita Fomichev](https://github.com/fm4v)). +* Tests: rename bad log names. [#66522](https://github.com/ClickHouse/ClickHouse/pull/66522) ([Nikita Fomichev](https://github.com/fm4v)). +* Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Multiple fixes for handling killed jobs. [#66524](https://github.com/ClickHouse/ClickHouse/pull/66524) ([Max K.](https://github.com/maxknv)). +* Allow GWP Asan allocations only when initialization is finished. [#66526](https://github.com/ClickHouse/ClickHouse/pull/66526) ([Alexey Katsman](https://github.com/alexkats)). +* Update 02443_detach_attach_partition.sh. [#66529](https://github.com/ClickHouse/ClickHouse/pull/66529) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Follow up [#66435](https://github.com/ClickHouse/ClickHouse/issues/66435). [#66530](https://github.com/ClickHouse/ClickHouse/pull/66530) ([Nikita Taranov](https://github.com/nickitat)). +* fix log in keeper tcp handler. [#66531](https://github.com/ClickHouse/ClickHouse/pull/66531) ([Han Fei](https://github.com/hanfei1991)). +* CI: Report job start and finish to CI DB. [#66533](https://github.com/ClickHouse/ClickHouse/pull/66533) ([Max K.](https://github.com/maxknv)). +* Update 01396_inactive_replica_cleanup_nodes_zookeeper.sh. [#66535](https://github.com/ClickHouse/ClickHouse/pull/66535) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add dedicated runner to libfuzzer, update docker. [#66551](https://github.com/ClickHouse/ClickHouse/pull/66551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* fix tidy build. [#66552](https://github.com/ClickHouse/ClickHouse/pull/66552) ([Sema Checherinda](https://github.com/CheSema)). +* No-op env change. [#66553](https://github.com/ClickHouse/ClickHouse/pull/66553) ([Raúl Marín](https://github.com/Algunenano)). +* Fix typo in new_delete.cpp. [#66554](https://github.com/ClickHouse/ClickHouse/pull/66554) ([alesapin](https://github.com/alesapin)). +* Fix something in Fast Test. [#66558](https://github.com/ClickHouse/ClickHouse/pull/66558) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* update trusted contributors. [#66561](https://github.com/ClickHouse/ClickHouse/pull/66561) ([Xu Jia](https://github.com/XuJia0210)). +* Delete bad test `02805_distributed_queries_timeouts`. [#66563](https://github.com/ClickHouse/ClickHouse/pull/66563) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* More clarity in the test `03001_consider_lwd_when_merge`. [#66564](https://github.com/ClickHouse/ClickHouse/pull/66564) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Stateless tests: temporary disable sequential tests in parallel. [#66585](https://github.com/ClickHouse/ClickHouse/pull/66585) ([Nikita Fomichev](https://github.com/fm4v)). +* Move view targets to separate AST class `ASTViewTargets` in order to allow extending it to support more kinds of view targets. [#66590](https://github.com/ClickHouse/ClickHouse/pull/66590) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix bsdtar for @nikitamikhaylov. [#66592](https://github.com/ClickHouse/ClickHouse/pull/66592) ([alesapin](https://github.com/alesapin)). +* CI: POC for Auto Releases. [#66593](https://github.com/ClickHouse/ClickHouse/pull/66593) ([Max K.](https://github.com/maxknv)). +* Fix clang tidy after [#66402](https://github.com/ClickHouse/ClickHouse/issues/66402). [#66597](https://github.com/ClickHouse/ClickHouse/pull/66597) ([vdimir](https://github.com/vdimir)). +* Adjust the runtime of some slow performance test. [#66619](https://github.com/ClickHouse/ClickHouse/pull/66619) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Scale down AutoScaling Groups from runners. [#66622](https://github.com/ClickHouse/ClickHouse/pull/66622) ([Max K.](https://github.com/maxknv)). +* Allow to run clang-tidy with clang-19. [#66625](https://github.com/ClickHouse/ClickHouse/pull/66625) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix terrible test @arthurpassos. [#66632](https://github.com/ClickHouse/ClickHouse/pull/66632) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad log message. [#66633](https://github.com/ClickHouse/ClickHouse/pull/66633) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Negative sign in prewhere optimization. [#66637](https://github.com/ClickHouse/ClickHouse/pull/66637) ([cangyin](https://github.com/cangyin)). +* Closes [#66639](https://github.com/ClickHouse/ClickHouse/issues/66639#event-13533944949). [#66640](https://github.com/ClickHouse/ClickHouse/pull/66640) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid generating named tuple for special keywords (null, true, false). [#66641](https://github.com/ClickHouse/ClickHouse/pull/66641) ([Amos Bird](https://github.com/amosbird)). +* rearrange heavy tests 03008_deduplication. [#66642](https://github.com/ClickHouse/ClickHouse/pull/66642) ([Sema Checherinda](https://github.com/CheSema)). +* Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* CI: Remove aws lambda packages from oss. [#66651](https://github.com/ClickHouse/ClickHouse/pull/66651) ([Max K.](https://github.com/maxknv)). +* Introduce separate DEBUG_OR_SANITIZER_BUILD macro. [#66652](https://github.com/ClickHouse/ClickHouse/pull/66652) ([Nikita Taranov](https://github.com/nickitat)). +* Increase backoff because with slow builds sometimes 100ms is not enough to recover. [#66653](https://github.com/ClickHouse/ClickHouse/pull/66653) ([alesapin](https://github.com/alesapin)). +* Fix wrong queries hung error because of 02044_url_glob_parallel_connection_refused. [#66657](https://github.com/ClickHouse/ClickHouse/pull/66657) ([Nikita Taranov](https://github.com/nickitat)). +* add log for splitBlockIntoParts. [#66658](https://github.com/ClickHouse/ClickHouse/pull/66658) ([Han Fei](https://github.com/hanfei1991)). +* Minor: Make `CaseSensitiveness` an enum class. [#66673](https://github.com/ClickHouse/ClickHouse/pull/66673) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bad test `02210_processors_profile_log`. [#66684](https://github.com/ClickHouse/ClickHouse/pull/66684) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix something around clang-tidy. [#66694](https://github.com/ClickHouse/ClickHouse/pull/66694) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ci: dump dmesg in case of OOM. [#66705](https://github.com/ClickHouse/ClickHouse/pull/66705) ([Azat Khuzhin](https://github.com/azat)). +* fix clang tidy. [#66706](https://github.com/ClickHouse/ClickHouse/pull/66706) ([Han Fei](https://github.com/hanfei1991)). +* Https://s3.amazonaws.com/clickhouse-test-reports/61109/5cf2b53f146c1a4f24d8212f9f810d587c46bfc0/stateless_tests__release_.html. [#66724](https://github.com/ClickHouse/ClickHouse/pull/66724) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* CI: Fix issue with a skipped Build report. [#66726](https://github.com/ClickHouse/ClickHouse/pull/66726) ([Max K.](https://github.com/maxknv)). +* relax condition in test, remove unused counters. [#66730](https://github.com/ClickHouse/ClickHouse/pull/66730) ([Sema Checherinda](https://github.com/CheSema)). +* Remove bad test `host_resolver_fail_count`. [#66731](https://github.com/ClickHouse/ClickHouse/pull/66731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `03036_join_filter_push_down_equivalent_sets`. [#66736](https://github.com/ClickHouse/ClickHouse/pull/66736) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad tests `long_select_and_alter`. [#66737](https://github.com/ClickHouse/ClickHouse/pull/66737) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test test_storage_mysql/test.py::test_joins. [#66743](https://github.com/ClickHouse/ClickHouse/pull/66743) ([vdimir](https://github.com/vdimir)). +* Disallow build exclusion only by CI settings (ci_include_, ci_exclude_) to avoid running builds in auto sync prs. [#66744](https://github.com/ClickHouse/ClickHouse/pull/66744) ([Max K.](https://github.com/maxknv)). +* Use non-existent address to check connection error at table creation. [#66760](https://github.com/ClickHouse/ClickHouse/pull/66760) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67063](https://github.com/ClickHouse/ClickHouse/issues/67063): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Fix flakiness of async insert tests due to adaptive timeout. [#66771](https://github.com/ClickHouse/ClickHouse/pull/66771) ([Raúl Marín](https://github.com/Algunenano)). +* Attempt to fix flakiness of 01194_http_query_id. [#66774](https://github.com/ClickHouse/ClickHouse/pull/66774) ([Raúl Marín](https://github.com/Algunenano)). +* Turn off randomization of harmful setting. [#66776](https://github.com/ClickHouse/ClickHouse/pull/66776) ([alesapin](https://github.com/alesapin)). +* The number of batches was reduced in https://github.com/ClickHouse/ClickHouse/pull/65186, but then the parallel execution was disabled in https://github.com/ClickHouse/ClickHouse/pull/66585. So now tasks fail with timeout sometimes: https://s3.amazonaws.com/clickhouse-test-reports/66724/36275fdacc34206931f69087fe77539e25bbbedd/stateless_tests__tsan__s3_storage__[2_3].html. [#66783](https://github.com/ClickHouse/ClickHouse/pull/66783) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove obsolete code from CMakeLists. [#66786](https://github.com/ClickHouse/ClickHouse/pull/66786) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Stateless tests: split parallel tests more evenly. [#66787](https://github.com/ClickHouse/ClickHouse/pull/66787) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix test `02724_limit_num_mutations.sh`. [#66788](https://github.com/ClickHouse/ClickHouse/pull/66788) ([Anton Popov](https://github.com/CurtizJ)). +* Better diagnostics in `test_disk_configuration`. [#66802](https://github.com/ClickHouse/ClickHouse/pull/66802) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02950_part_log_bytes_uncompressed`. [#66803](https://github.com/ClickHouse/ClickHouse/pull/66803) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better diagnostics for test trace_events_stress. [#66804](https://github.com/ClickHouse/ClickHouse/pull/66804) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make test `00997_set_index_array` lighter. [#66817](https://github.com/ClickHouse/ClickHouse/pull/66817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#66818](https://github.com/ClickHouse/ClickHouse/pull/66818) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix inconsistent formatting of lambda functions inside composite types. [#66819](https://github.com/ClickHouse/ClickHouse/pull/66819) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Never await in CI on BuildReport - just redo (await can be longer) - Remove BuildReport if no build jobs in workflow (for instance: Docs change) - Do not fail CheckReadyForMerge job if the only non-green status is Cloud Sync. [#66822](https://github.com/ClickHouse/ClickHouse/pull/66822) ([Max K.](https://github.com/maxknv)). +* Remove bad tests @azat. [#66823](https://github.com/ClickHouse/ClickHouse/pull/66823) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: New Release workflow updates and fixes. [#66830](https://github.com/ClickHouse/ClickHouse/pull/66830) ([Max K.](https://github.com/maxknv)). +* Fix signed integer overflow in function `age`. [#66831](https://github.com/ClickHouse/ClickHouse/pull/66831) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix RocksDB bs. [#66838](https://github.com/ClickHouse/ClickHouse/pull/66838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Split a test for index. [#66839](https://github.com/ClickHouse/ClickHouse/pull/66839) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix inconsistent formatting of `NOT ((SELECT ...))`. [#66840](https://github.com/ClickHouse/ClickHouse/pull/66840) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make test `01592_long_window_functions1` lighter. [#66841](https://github.com/ClickHouse/ClickHouse/pull/66841) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* What if I will change the test for SSL authentication?. [#66844](https://github.com/ClickHouse/ClickHouse/pull/66844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Functions `[a-g]*`: Iterate over `input_rows_count` where appropriate. [#66846](https://github.com/ClickHouse/ClickHouse/pull/66846) ([Robert Schulze](https://github.com/rschu1ze)). +* Revert libunwind patch. [#66850](https://github.com/ClickHouse/ClickHouse/pull/66850) ([Antonio Andelic](https://github.com/antonio2368)). +* Split test 03038_nested_dynamic_merges to avoid timeouts. [#66863](https://github.com/ClickHouse/ClickHouse/pull/66863) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: Print instance info in runner's init script. [#66868](https://github.com/ClickHouse/ClickHouse/pull/66868) ([Max K.](https://github.com/maxknv)). +* Backported in [#67257](https://github.com/ClickHouse/ClickHouse/issues/67257): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* CI: CI Buddy to notify about fatal workflow failures. [#66890](https://github.com/ClickHouse/ClickHouse/pull/66890) ([Max K.](https://github.com/maxknv)). +* CI: Add ec2 instance lifecycle metadata to CIDB. [#66918](https://github.com/ClickHouse/ClickHouse/pull/66918) ([Max K.](https://github.com/maxknv)). +* CI: Remove ci runners scripts from oss. [#66920](https://github.com/ClickHouse/ClickHouse/pull/66920) ([Max K.](https://github.com/maxknv)). +* Backported in [#67209](https://github.com/ClickHouse/ClickHouse/issues/67209): Decrease rate limit in `01923_network_receive_time_metric_insert`. [#66924](https://github.com/ClickHouse/ClickHouse/pull/66924) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67227](https://github.com/ClickHouse/ClickHouse/issues/67227): Grouparrayintersect: fix serialization bug. [#66928](https://github.com/ClickHouse/ClickHouse/pull/66928) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67207](https://github.com/ClickHouse/ClickHouse/issues/67207): Un-flake test_runtime_configurable_cache_size. [#66934](https://github.com/ClickHouse/ClickHouse/pull/66934) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#66975](https://github.com/ClickHouse/ClickHouse/issues/66975): CI: Fixes docker server build for release branches. [#66955](https://github.com/ClickHouse/ClickHouse/pull/66955) ([Max K.](https://github.com/maxknv)). +* Backported in [#67213](https://github.com/ClickHouse/ClickHouse/issues/67213): [CI Fest] Split dynamic tests and rewrite them from sh to sql to avoid timeouts. [#66981](https://github.com/ClickHouse/ClickHouse/pull/66981) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67033](https://github.com/ClickHouse/ClickHouse/issues/67033): [CI Fest] Fix use-of-uninitialized-value in JSONExtract* numeric functions. [#66984](https://github.com/ClickHouse/ClickHouse/pull/66984) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67051](https://github.com/ClickHouse/ClickHouse/issues/67051): CI: Fix for workflow results parsing. [#67000](https://github.com/ClickHouse/ClickHouse/pull/67000) ([Max K.](https://github.com/maxknv)). +* Backported in [#67116](https://github.com/ClickHouse/ClickHouse/issues/67116): Disable setting `optimize_functions_to_subcolumns`. [#67046](https://github.com/ClickHouse/ClickHouse/pull/67046) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#67205](https://github.com/ClickHouse/ClickHouse/issues/67205): Increase max allocation size for sanitizers. [#67049](https://github.com/ClickHouse/ClickHouse/pull/67049) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67124](https://github.com/ClickHouse/ClickHouse/issues/67124): Very sad failure: ``` 2024.07.24 13:28:45.517777 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} executeQuery: (from 172.16.11.1:55890) OPTIMIZE TABLE replicated_mt FINAL (stage: Complete) 2024.07.24 13:28:45.525945 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (ReplicatedMergeTreeQueue): Waiting for 4 entries to be processed: queue-0000000004, queue-0000000002, queue-0000000001, queue-0000000000 2024.07.24 13:29:15.528024 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e) (MergerMutator): Selected 3 parts from all_0_0_0 to all_2_2_0 2024.07.24 13:29:15.530736 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Created log entry /clickhouse/tables/replicated_mt/log/log-0000000004 for merge all_0_2_1 2024.07.24 13:29:15.530873 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to process log entry 2024.07.24 13:29:15.530919 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to pull log-0000000004 to queue 2024.07.24 13:29:15.534286 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Looking for node corresponding to log-0000000004 in node1 queue 2024.07.24 13:29:15.534793 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for queue-0000000005 to disappear from node1 queue 2024.07.24 13:29:15.585533 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} TCPHandler: Processed in 30.067804125 sec. ```. [#67067](https://github.com/ClickHouse/ClickHouse/pull/67067) ([alesapin](https://github.com/alesapin)). +* Backported in [#67203](https://github.com/ClickHouse/ClickHouse/issues/67203): Fix flaky `test_seekable_formats_url` and `test_seekable_formats` S3 storage tests. [#67070](https://github.com/ClickHouse/ClickHouse/pull/67070) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67222](https://github.com/ClickHouse/ClickHouse/issues/67222): Fix 2680 flasky. [#67078](https://github.com/ClickHouse/ClickHouse/pull/67078) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#67190](https://github.com/ClickHouse/ClickHouse/issues/67190): Attempt to fix flakiness of some window view tests. [#67130](https://github.com/ClickHouse/ClickHouse/pull/67130) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#67272](https://github.com/ClickHouse/ClickHouse/issues/67272): Rename (unreleased) bad setting. [#67149](https://github.com/ClickHouse/ClickHouse/pull/67149) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67441](https://github.com/ClickHouse/ClickHouse/issues/67441): Try to fix 2572. [#67158](https://github.com/ClickHouse/ClickHouse/pull/67158) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#67416](https://github.com/ClickHouse/ClickHouse/issues/67416): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 271065a78fb..027b207d3ad 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.4.49-stable 2024-07-01 From 8d2b804c670d0941acc4fff059859017c1bd93c2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 21:57:08 +0100 Subject: [PATCH 246/496] fxs --- tests/integration/test_recovery_time_metric/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 8f369d7759c..628f2e744e0 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -37,9 +37,7 @@ def test_recovery_time_metric(start_cluster): """ ) - node.exec_in_container( - ["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"] - ) + node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) node.restart_clickhouse() From 4aedb9d40298c1a3204bb72a3288ea711eb5e2f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 23:02:22 +0200 Subject: [PATCH 247/496] Update test --- tests/integration/test_system_flush_logs/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 2022f9d4a89..713b327eb76 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -13,9 +13,8 @@ node = cluster.add_instance( ) system_logs = [ - # disabled by default - ("system.text_log", 0), # enabled by default + ("system.text_log", 1), ("system.query_log", 1), ("system.query_thread_log", 1), ("system.part_log", 1), From 4a4bd97b4b63e495f76273ea8c12045dc129d81b Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 16:50:37 -0600 Subject: [PATCH 248/496] Fix case sensitivity for percent_rank, dense_rank, and their aliases --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a1b46c8e36c..5fad68e4968 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2726,18 +2726,18 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) From 3b12fec141fda8cb2a3ef68ac96e6e58f1fd69e3 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:05:48 -0600 Subject: [PATCH 249/496] Update dense_rank doc to mention the denseRank alias --- docs/en/sql-reference/window-functions/dense_rank.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md index d6445b68c55..2c8617fb668 100644 --- a/docs/en/sql-reference/window-functions/dense_rank.md +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -12,6 +12,8 @@ The [rank](./rank.md) function provides the same behaviour, but with gaps in ran **Syntax** +Alias: `denseRank` (case-sensitive) + ```sql dense_rank (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] From 419a5e7f730dabe514becabc6c24ec5b87325e28 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:17:01 -0600 Subject: [PATCH 250/496] Update window-functions doc with denseRank and percentRank aliases --- docs/en/sql-reference/window-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 0c3e2ea1cb6..27d4bd763c7 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,8 +23,8 @@ ClickHouse supports the standard grammar for defining windows and window functio | `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | -| `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`| +| `rank()`, `dense_rank()`, `row_number()` | ✅
Alias: `denseRank()` | +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`
Alias: `percentRank()`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From bc312eb046db07d901e208cdc1bb0abb1df3eabd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 21:27:50 +0200 Subject: [PATCH 251/496] Improve check --- src/IO/S3/URI.cpp | 32 ++++++++++++------- src/Storages/StorageFile.cpp | 18 +++++------ .../03215_parsing_archive_name_s3.sql | 2 +- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 33a4939c810..fead18315d8 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -55,10 +55,10 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) static constexpr auto OSS = "OSS"; static constexpr auto EOS = "EOS"; - if (!allow_archive_path_syntax) - uri_str = uri_; - else + if (allow_archive_path_syntax) std::tie(uri_str, archive_pattern) = getURIAndArchivePattern(uri_); + else + uri_str = uri_; uri = Poco::URI(uri_str); @@ -176,22 +176,30 @@ std::pair> URI::getURIAndArchivePattern( return {source, std::nullopt}; std::string_view path_to_archive_view = std::string_view{source}.substr(0, pos); + bool contains_spaces_around_operator = false; while (path_to_archive_view.ends_with(' ')) + { + contains_spaces_around_operator = true; path_to_archive_view.remove_suffix(1); - - if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) - return {source, std::nullopt}; - - auto archive_uri = path_to_archive_view; + } std::string_view archive_pattern_view = std::string_view{source}.substr(pos + 2); - while (archive_pattern_view.front() == ' ') + while (archive_pattern_view.starts_with(' ')) + { + contains_spaces_around_operator = true; archive_pattern_view.remove_prefix(1); + } - if (archive_pattern_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Archive pattern is empty"); + /// possible situations when the first part can be archive is only if one of the following is true: + /// - it contains supported extension + /// - it contains spaces after or before :: (URI cannot contain spaces) + /// - it contains characters that could mean glob expression + if (archive_pattern_view.empty() || path_to_archive_view.empty() + || (!contains_spaces_around_operator && !hasSupportedArchiveExtension(path_to_archive_view) + && path_to_archive_view.find_first_of("*?{") == std::string_view::npos)) + return {source, std::nullopt}; - return std::pair{std::string{archive_uri}, std::string{archive_pattern_view}}; + return std::pair{std::string{path_to_archive_view}, std::string{archive_pattern_view}}; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index efb39f90053..8c079aa4600 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2258,21 +2258,21 @@ void StorageFile::parseFileSource(String source, String & filename, String & pat while (path_to_archive_view.ends_with(' ')) path_to_archive_view.remove_suffix(1); - if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + std::string_view filename_view = std::string_view{source}.substr(pos + 2); + while (filename_view.starts_with(' ')) + filename_view.remove_prefix(1); + + /// possible situations when the first part can be archive is only if one of the following is true: + /// - it contains supported extension + /// - it contains characters that could mean glob expression + if (filename_view.empty() || path_to_archive_view.empty() + || (!hasSupportedArchiveExtension(path_to_archive_view) && path_to_archive_view.find_first_of("*?{") == std::string_view::npos)) { filename = std::move(source); return; } path_to_archive = path_to_archive_view; - - std::string_view filename_view = std::string_view{source}.substr(pos + 2); - while (filename_view.front() == ' ') - filename_view.remove_prefix(1); - - if (filename_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); - filename = filename_view; } diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql index 3a7ed0b864c..e34be475c5a 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -2,6 +2,6 @@ -- Tag no-fasttest: Depends on AWS SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); -SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } +SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError S3_ERROR } SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); SELECT _file, _path FROM s3(s3_conn, filename='test.zip::03215_archive.csv') ORDER BY (_file, _path) SETTINGS allow_archive_path_syntax=0; From e664a144788b48c029f56548242baaeed82a80ff Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 08:49:14 +0100 Subject: [PATCH 252/496] fix style --- src/Databases/DatabaseReplicated.cpp | 4 ++-- src/Storages/System/StorageSystemClusters.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 06cea65d62e..b2be593d326 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -356,7 +356,7 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); } } - + try { auto current_zookeeper = getZooKeeper(); @@ -396,7 +396,7 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) ++global_replica_index; } } - + return replicas_info; } catch (...) { diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 4b9802c9089..d03b600b6ef 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -116,7 +116,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam else res_columns[i++]->insertDefault(); } - + ++global_replica_idx; } } From 5152248d438ef9162845507b68e18f1d8541a250 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 09:25:59 +0200 Subject: [PATCH 253/496] Add test. --- .../02864_restore_table_with_broken_part.sh | 25 ++-------------- ...ackup_with_matview_inner_table_metadata.sh | 25 ++-------------- ..._clear_old_temporary_directories.reference | 2 ++ ...kup_and_clear_old_temporary_directories.sh | 22 ++++++++++++++ .../0_stateless/backups/mt_250_parts.zip | Bin 0 -> 265998 bytes .../helpers/install_predefined_backup.sh | 27 ++++++++++++++++++ 6 files changed, 55 insertions(+), 46 deletions(-) create mode 100644 tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference create mode 100755 tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh create mode 100644 tests/queries/0_stateless/backups/mt_250_parts.zip create mode 100755 tests/queries/0_stateless/helpers/install_predefined_backup.sh diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index 08313e2fd3b..bf76727f76f 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -5,29 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, -# returns the path to the backup relative to that disk. -function install_test_backup() -{ - local test_backup_filename="$1" - local test_backup_path="$CURDIR/backups/${test_backup_filename}" - - local backups_disk_root - backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") - - if [ -z "${backups_disk_root}" ]; then - echo "Disk '${backups_disk_root}' not found" - exit 1 - fi - - local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} - mkdir -p "$(dirname "${install_path}")" - ln -s "${test_backup_path}" "${install_path}" - - echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" -} - -backup_name="$(install_test_backup with_broken_part.zip)" +# In this test we restore from "/tests/queries/0_stateless/backups/with_broken_part.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh with_broken_part.zip)" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" diff --git a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh index 8d987dbf1df..2c70cb1e3be 100755 --- a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh +++ b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh @@ -5,29 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, -# returns the path to the backup relative to that disk. -function install_test_backup() -{ - local test_backup_filename="$1" - local test_backup_path="$CURDIR/backups/${test_backup_filename}" - - local backups_disk_root - backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") - - if [ -z "${backups_disk_root}" ]; then - echo "Disk '${backups_disk_root}' not found" - exit 1 - fi - - local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} - mkdir -p "$(dirname "${install_path}")" - ln -s "${test_backup_path}" "${install_path}" - - echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" -} - -backup_name="$(install_test_backup old_backup_with_matview_inner_table_metadata.zip)" +# In this test we restore from "/tests/queries/0_stateless/backups/old_backup_with_matview_inner_table_metadata.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh old_backup_with_matview_inner_table_metadata.zip)" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src" diff --git a/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference new file mode 100644 index 00000000000..3f3fbd9ab58 --- /dev/null +++ b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference @@ -0,0 +1,2 @@ +RESTORED +250 31375 diff --git a/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh new file mode 100755 index 00000000000..e0c8f08e695 --- /dev/null +++ b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# In this test we restore from "/tests/queries/0_stateless/backups/mt_250_parts.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh mt_250_parts.zip)" + +${CLICKHOUSE_CLIENT} -nm --query " +DROP TABLE IF EXISTS manyparts; +CREATE TABLE manyparts (x Int64) ENGINE=MergeTree ORDER BY tuple() SETTINGS merge_tree_clear_old_temporary_directories_interval_seconds=1, temporary_directories_lifetime=1; +" + +# RESTORE must protect its temporary directories from removing. +${CLICKHOUSE_CLIENT} --query "RESTORE TABLE default.mt_250_parts AS manyparts FROM Disk('backups', '${backup_name}') SETTINGS allow_different_table_def=true" | grep -o "RESTORED" + +${CLICKHOUSE_CLIENT} -nm --query " +SELECT count(), sum(x) FROM manyparts; +DROP TABLE manyparts; +" diff --git a/tests/queries/0_stateless/backups/mt_250_parts.zip b/tests/queries/0_stateless/backups/mt_250_parts.zip new file mode 100644 index 0000000000000000000000000000000000000000..15310b79054e09c75876efd20845bbdabdfec2f5 GIT binary patch literal 265998 zcmZ^LbyU>d_qBycBi$g~(%s$NAT23KcS(1Hv~;I1q$7e1-7V7HNF(_UqxgN-`pxq% zi?u%M-gC}A`|PvtD9Jv3hVkghix-bx^WJF%st<_?m_2%wHSzor?4w7I7>$igtex#v zfqS3Ho1zSV(_TH{Cl!!FHF@ii#D+%V3LtZC0OSLHxL*cIVv#w>nQ$yQ9NY-Mexva= zsnm9PanzoGNa2X{s=+tY_fqig;^!Uf?MjvJWrObx+4Wh-Y2Sj^#lqdqkniWShC`Fv zt)Z2wuDh$4yI&2hH(Rg4|7q>}`&`gitX1627ys_E;X*!oVJD{b#`>!7BCnwp z5NcB2diVRi??vB&kD-F#E%2sm;r24*Y(wzQFHxWDrtP-M=x(pd_pZ<2u5aP?YQg8~ z`%dc(_%D~lD=|{P8w9`joc_H0cq=tm5Ofjw6)EO=o5_KTWB*C!Q_VN}HsCysUt9*{ydw3%9QbE_VLxz9JxlSYTvsU!JC2` z@GTqMuHN1FMi*U+3KeMYb9n#?$IN4@M!K6x9wYzqd~&&S5hDl?yxa1vYbCV2Y`wdD ze|ISeQa`>UtsIf~pxxGH{UOCPg}U%=IGH5MbNTY%t^oYc$UzsxB3q#WZwIY7$}x*8 z>xCR;f{*Ji|zH3p5wOmh>kr;4%G9~9dp16!nfsKEx!}JTUXy@%xiBN8V z@O%3F_lUc8gu%JRn7{d{`_=AeLkwC54`Gi>FR6bLJrB z?06B@h6;tTG{nPhNw6&-;xrJJoAH~7rHb0yWYb|svt1P#wW!g(EaMs=EcT`*QR4jb z33_?etnsa_>lnFR;mqr_gI6Ka@eBv(GYQ2t+^52C~kX_T-~!NUptBc=wE5O zX;fD5-bOq=9S6~H4uz?xZ8<5{OwR`(oY0{tR0^n(3~_i3W{|BI?sApK*UlvzNAaZ? ztkl6u_HTxdnhiVxM5h*}+0|IIRJ^&SKPMdXA-c}qTWu3RRYGCBrgF2Z$@*mox)@nE z)o~*KCek!zl*cp@v$PDsA~?*Nm^z8Kwb8%{>#xx#(UyWJKEp0Yi@6@fu-Lcge2rR)Zm}I=>~Zc+ek^5f_3%-; z7&|I5-1Gs1j}XwC*Dl;jeTO=-6q> zlAWXGomf0ILXOomjq=rGW+;HTE|agZm}t8#=dN%_+fL zpsgBb5);jPpT;p&wODG~0Esr^_+_BW@bg;E@I+1pFZ-eJ1+tl;4+!t5p5B^z6yN!@ z3%2r*<`g;Cx7{9}W^C}o1Y7&*t z4e>_ve`8;X-)|$tZi~avU{-N@L87F(lT6ty592|GvTU8@EapAt?Jz7MXb*9hAfGI_5eFaDaWW<8ScVASYR4 zAM`*C6Q*+rOGloHHO!wiFqb_snIdXRUWW!R>xmRyS=zi-P9Kop4fOVlmqro@PN)9? zF$4E%zlMw${!gkqzy=k3ipX1CTF$SRUpMdT9*i#@*ns^_7~N2+M`Q0Q3iYk~KZQ|a zyH4D8)O~^#wBU8OQ|2EJK)8b=(km}=L7y1kt_eY;6^R#lRB@V!$uuwyPUD}Rb{~A! zggYXY;d_K4= z#78&{A;@{$w=1i&HJM`D-}cV}-ZDM`oV#e=!z+hMMLe2or<8 zv+lW5OF-8w%A!|OT^<7N$5m;H-ZHGg8rq%ZfrwbY316kI+u0;_ zP!;a=pCimj?0c&6kNLd6hP^l#@Gv7Dk&&3lGhOV?(slAPQ8}T|w(AD5x-2~2+yBP7 zsfktIOHqg}8`jZLW*Hn>cNg=1@EokBrYfQNor5&um`60bY8&6AK}&7`Zj@taz(S>rtdpx!qa zigkrD8kyR&Rnl+@G?bADPC}+ipw~&*|7~PLyQOnS{ftc7IZDAgrDwK=Bl#A3+5bq) zV%aT#Jnl%!am3=gc+p2Sr&8_a2b|eLORk`jU{WUg9vJ7w2=Lg)-2q7wLgyDNu)M4% zje=$|pZPwkNR-YyoyDa+RHtoHhiMVKnS0?Q0^>)kYRgvJWGi3(vY3;7W)M6vEGOP1 zDSd_b9vXEa9ME7Ebs5@){ot1<#FBZ+M#~h%BJYse{#{yjJo@3nyalGi`%^Rny4G04K1|vYVd}_R>rG3oEoOA9}hlq21R)N907HGK)hk_OJTlP&6LW6}Q zYEzV4f0oL}2)e2H$2RvZd5#vYVFY#E{>#L-c=pF-sPzP4#7*<}WJyaO9F|v}{flZk zA34xn6rVarv*7}s=v{AKK_g4r>{aA!%x@>+d#Sgyq}L8@(oC`_9{5FXK;T>C=e5EG z`4^AJ3pr$8k2D}MaAIyoK~RQMd1ZXMprJ7`X*v~O%w#jsv&;U_IUgFDnN9EwScoFl z5M5~n-^gU*26`FF!vB9VD zCh2%JU;A5y-FKe{(l=-NqR4KKEAKqoVKYw(Z!qtS+~>Z@5E+;=XuW)=Vddi% z@^MmAcEt(Qa zVxRbxXMCUf<kLe*G2pqQ?>x^iB*=Q$S9zJIp9vxHHRRaJrpa-5iG*v(c@x<8Pbc z)vb1|n(#n6CU_sBfs=eXeclv;@M__{YNYO2xdIz?7r(?dE2Ad!(e64>nuc&4#MTOh zEVH&)p=+cd0h91FSH5lib~7TQFAk2~|Jybjbd(UzW;{$j0i=5Ks>uok*e;LC&WS$V zpQg2QYtAHk>Pv48%Ba#PCIY$rtY(9Y|_Gdmun&m!U^trfsW~K#~76>w8 z_E~3~8NlV;XnpKL~`ty07UnHaP(cXnJegA6# z8MW%)w?*qjyi~(j6ln9-OFd=->g%Gj*);Nn$`N!&X$n-;K*4iu-@~aDs)(=0y=6_Gptntabp|EDwmaa%1!Vjq>xJ8M1??{-F8wDarXa7?3 z!GMD1!*^n&Gv@wNORmpH@>cb}K(M(O%*_aaY~9&;iD@gPQIoL|VzS)@Y(Ztj-=WQ9 ztV7~-BZkH<=kGu-95XyA?IKE zYBn7&`gy&pZxtv@gyC}7$d+!^k|*}={nt+^3wybgJr`UXhHl&9+}u|1J8UwR$Z4*t zp2R!$9#OUPJ0Hdgj^uLED{E2qOPq*RLM2+ya(19{n!>9wkBsb&VX9i`QY|kR3%;>K zO)x>x$)Kk+F#DHtffgY(>e&dzqmzDyuL?ix2Rg8M4l8QF+gt=m+juD zABXglz6{1Jh6`w1BZGmu%QMG07AM3Y-pqzSHEt|XWA&ey;WX*5##)~oGr(|M(*kF) zpB?m*9>O-klhZyd86f!yOtZ3$48JW$=_JvV?Y7t6<2z_GO|56Yl471BD0jXl)lRxZ z-LW%e(C(oZO!(J6CX5kB7_?2w$<0o5$Php=ga{2Jz)p7*P!bB%L^1~_PfY4)N^!b}<|egw5#4tpWuH#DH(>$Wu8;?o(TVU#T8BMA_9 zj5kvH*FAAts10v6uU`6GI%R%l)9PVW&yS~74~j1Q{!L1a z(f0gWminh#^%rn;no^=5x7HE45*B;#oxG;dIoze$J^^jyLL-9XmH~@%1oFiq?^&he zuriktd5#(z>n;d~|B67NUIcF^^8HUrLFoSZGtuA*@=^thK-piwvYbd+$xb2>CpMg+gDk%)6=gCtGUz(~l@HuihYd8l8BwhgAC2|>X{ zqPNWiHPs>opP8wz&avKnYlIjI6=9q9o3n}-pmg)y4)Lr#&0exi;9IImj_hW z=)30$Cy`f54E{o+&nBe9t*}N%=-4t%T@yq12F}rsTgKKYhN$6FOe0@bbbliP8Y~re z(#SYAku6+ly}sSYF$ZeO!Ue9#s^{J@l`GYJ zEFQGwK|8r;X>%Xm4_DE1(S~|l$0eCXc%l+3t+;#_pRkLD+}RSCq1(ki>B^_1jS}_2 zvQ)4aHYYlE`|@A?%x7l0mdC|HEbC>_AWKQ^s;PZW^50}Oa7Wd(>7r|7`K;KNq4U~y z)+w|I`4qyOyrRb5hsdNehp0Ygi1*l^F0nexSABFb;|Qi`|5s;R52F@i#es>9%@uI^ z_=H_k<7p_o$Uo}bASxYRLK+ZJ?OqZfi@8&4$0Y|D0rRK0cG?IEd&WEySGa3B2_+I= z%O--EMr$MUDAc+XK0=i>tJTVoV2%7p9Y0{sD}d;nCf2s#g( zMSytfCn(~4zoW8fiZQ^Lj>f2uR&+W+w*Hy=|45*o&L`H5sayS69sJ^fTq~5+jzGw* zO-^y`2_MT)a=%-_EL|If_wHZuDjPpE{&@q1Hz`(RD+;`D2*?PRH21f^>(_Eh%T|97 zfZqs8eZT4_{He_P$j(fdGj6lD?RAzD1euk<_}?5R@FFR#qoWd#v?5m)zg{CuqS(2t;@)l-TnBTQ$&D=}-I zf1TxX>&XR~p^~9`ZVG2S->qgEt?)+82G$X}ps=s#B&a8YU~^?ao$$$vDpf zB~E8be?vop&VYf`(vw1D?qZJ(jkW0@`$3K7xra@_1mACxw!%1D^LLvSn!=Hlv9OL1 zoffRpTo-*vp<4T)7D1$5hEB68Fl?9TVQ&R}t_jmHlFA=@SEh z&eVW1gLxf@fepB=rq81XgPgY^AaH_Lq^K_Q^jn&mNFc2Mf-63tHuMM16o&-Vk!C*p zvD1sH$P$`w)r547<}#`i$5O`+wRy9+YxIq+{Nv`A5{*9i(Ts5O*Vsd8clS&KOlU5T z+p-El&a_SatC%x~BXk3ka;oE15e;t8C_o@ALRnsioB&a*L&Z3^WKb4UZ=L_zztgZO zl(T!Z>F`M-6FoVcUr8o+e+_}>wrji%&4G`l_ARfnk$EJqo4D8t#&MIXDzL7h@g|Ld zkZ0+J7iL6VI})m!@ri|LC}Gmx4_3@?TbEOf ziH0H!D?U}Hy?Gh! zC2d`Ht2q7#m=?084to_tOTZ7O0x|NtM+Bj)e>f8uV-&^nfRoIs)<*3aTn&6H`cKPH zKSpDx??c@+DK@@Ni%We=$-w{Af6X#ps3Pn{?Iank2Kd{@*k=tmT9`zQA>K^&Hq#;n z4Us;~y?ORt2&UR%AFRSkG!nOR@&TxyGwTspi-ZJn8}K%RH)t;FS1>kY=fILI%T~_Y zr`P9Z{9`wuhOuIQLdwLH9?Cbbl-a0KiVSA$?g)qXk-Y+Z8bUavib{Lvm`$)!yM9#J zrkRxh{;V9pD{oh2Itf5*E$Jhysd7J{`ilbsMefRi-_RnXO%W&aiBzDbw~iXu=}V<@ zu7=%4;_-t9kmbJ~7gI6DLWNl)<=9V4Sb<2u+y@!YJHXA^G8-TEB{K0L(htOC57`0F zVD8u|VBi(B(TwlqTGnzQH;OT-Cy+HZ2zbhvtNQz2Rc6c^Zr7AZZvm2cnpQ2onOh^} zk|aa88CX3(AY#o4b(T%OutSQv&}Ho(@$5qC0-!WQ6`YQf0&yG(Y1VCLIYh7dx(<1S z(YC~*9+IFT-D0T6arMnuf2`+hm|TGXoZmMP1e907%t_crjn!}~Lq@zZ|IY8(dceyf zq-F_mdX^>((@)Q7X(V?>r)67bP<%vwbOx-kD|l9GPv9M2xjmh6Dg`~0!8>=)GYbAM zg>KGdh9s2%KHzND;eGZ@_~*$C-d7KbMsGlBbWCM1ugZ87pNP0puAZk0579DYIZq9S zS1Xx{KDT|(nb?gn**qctAR8|^H9^ra-7a)YT7YY+8* z8bIJm$leuXV#aTV7Qs7Rr)>B90McV1<4murYQTt$d{VwbhITYMe?B(se=X_8Hn?4% zXK$yr|KjoWG%49DEa;F^AQ{*Jku%as&Hn`Yx_APw>G4CbF+9Z0*fZR$2b1< zaaZ$dLv;)IcaLv9YqOXQ7vVGyqJqZqhY<=zx^9z4;^e>-@Q*r!bwcEff_FLGm{l6n zigIcufCaGOq^9do^@mrX(zG!nBeS1`_I&n4lxCep?! zL(5;O&MNQyu%Ow0CoEQ`MGybViZrbP$@pnE-W1jer1}rR(<~-ByVkl>2f;aZ1eDx&O-;ux)E|(Z2xi5k zA$GIwf6O7_93c3(_W+rLTE-bA+M%y?Y&jFsJUQvKT9S%MAUSYko#FesQ{)3kC(0j@ z``edj4xq!fm`W9NVE$!T<{r)BoIKk1C?Ib6{kAEdGWP!tGZ-{33pYiafm_ zAoua&AN$X-b;g2t#p%3Cw;eUsO4hRL;tKaxjrW^Yw}^9S1ArSUT5@tCDr@4iVsaog zmE3&EWctItnMp1{IogJUvu4Gw46&<7OPa@_$Kk;>QwD?1&JwP}Ctk`T<+E2d36E4_ zXXEG(HTsDKn{tunxO0*#e4%)Saj3tzvO2H+URV`hXuftG9;X6~P;Y)%*1&Vq4}1_v z1`EJD!hJnDM8L-^Is?5h7hK$k&Pre18V{ibr8Xi>YMg&tUZpOetMe+SO@RMWgyD2~ zF!|t5^li^^a-}ssL<7*~!Lxsn9hnT4E7>sQL*RYWWB@L8j1fbj;!tKUB6_%=+$b)0 z>qYSjG-Td_EmJMkHlCRb$3+9k5wSCczd~64cNnhlBmp?duufFN`jN89cR=^v-4)3E z?+unRS9z!jJ&(1_&7M5(dIoeYsrU^FP=zKj;i9N)`~ZCHMHb3Z;h$e(${b#AY)-cM zD*+FQ+xRD~i{VDFQbm#CsM=tKv}-QNUj}}Tx}TTKWfvFqdDK*U02>(fG{YVok=H*# zAevUmx=&BBrxl)G_nau*-30pj-}uC5*?sb0J?05ORej4XkVJZ*49i-J_s=$?@Z)Vd z4;2|!Wy+Vyj7LT%J)`BbD)HwvS%u)x0JoYbY)JATG+hnG=TYmd0ru3%2;$T)PiOupGdS3Tei9aT??}F{Ye>*hE5CYmndlBFOz^<` zW;~j|E`pMKf1^lwxgv@Y|@(230bWH%7muZcufD^>!2u~F%vFS`<* z%roqBJn!E`UI+$+Y>VztG;R0g~VP9;f@ zkd*2Q_LEK|aU{ppkMD#{@)oeO7jiRLZCiM8s=vP_4gFV zXx{C=H)i0@w9nT}K}<`CS*#wc)0-c;&o>5fG~CNFwz>NNX&wQDx~)`|)aOW<5)<)N z0zS}~3>9Cf^6>lyb!y z>|FcoHd*(_?72k#KAfF1_Te)vN+Gs8v)sr4bVgOz$#{!o$-xyRPa-DHoVHiDZ1{Kq zdF8fdn#Epa1*#imTV;w{IpDlXb>N@+$%VYaYsAy4x>-}1L^dQ-^=32pP--mi=dno~ zo!(6F^~V$!$GIwttBZZnQ>hK_U+3rZA*X>?xhFuD)^ZjFkui&`hvpp)jEy6w5D8HBS<~NExj5AXRW~`>%`p)(zaFZTtNaH05hKZ0> z!~Oa^$*- z5rj=Rd}hwSbYcun#&|BBzNeE0T>v=ZA|F%10}fsQFdAGkaxvz@wHhf5JQ-;T z1b<7Ij=wObjZ5gQ5l%zPxBMTTm0@t=0;AQ0fP=J|%kxjU*IlYA{$p?M7{6Fu?7rt_ zJ?nse`TMA|e@2zD;M+eAP)wi2a2{9vQUW`gB@gz!a-&VM800(2U=aBHg*)93vocUs zZdh4Y9Q}A^b9t{5S_fEayMQ~RPjj>_P7Z7jL??HKzY`*Qm}ZVhC@|q+@sf=zC`zC1 zQ1?)*Ykh?58Sgdb(+?o(2@9AB*fsKaQ@7-laoQ1Q124iYXkd0^_>vy|F@-2MWu-%6 zn21;`|ARpjOw7CpKB_rps+jTWPo1s@JI7OWA4AqZkcr8PuCOb-Uya-KS@-e?A!V73 zq4d<~HB^pihvxYb^H<($OLbV{_n5Kg`)WrIBJF{|AAz_ zcv7d)_5lPra+$$hWq9~q{UK?Y*UD;w&PgZlH|FjJVMXB1gWF7-t$bz zWcC!SZIk*mVRV+Jwjsx{QrZM2}mZ z%!B{q*9SA!aESm7-~^n&g9T&?rM}TL)UmJjX+b!^*!ED9TVc}qkEI!$XI5?V0nZaX zs20xzzi=~!O~dtp=+!{U+s6)MP!VwJj?zx%oT)qUwu%%p4-kegS$ zZ;DB!#p(COpE+i!K}L*zh|$Prcc_<#Ce8upg`cpNyD`%Ge<%!>!mRWO>O4r%pDdDA z0RBvdAA6lJ4tf8+WvECyU~#}VHLR_jf*&wC@62mzvv9|~gWa1Idui;f zv&|k121&5iw9?|Z=Ka-;u+rr00O~MNNj*b|&#dhdZ$nXA#$P)nY9};WS0Ynf*zDAD zm=XGGi6J+;V%7kHQ)A-|J&%dh@Xc(-!H3%aDQ;LxC6xJi~K5o#fHQQ$OE9V63-g&35-`#w*BhIf6}u5+G+vgKxvj=g}>mt+Q`Z#aRmIsM5YZE zF>z5HDLOX1MRwkjOT*wxH>6hDL_)X!60N9GZ`=3Jz^BN>#4pj3mJstQ4g912@hNaT z3Zf}vbbgm{?yqU!O=siAA;CEW@8p@tuo;2}WS%}D&=|w9RI0bN(UXH}V>=bAJEKNk z-1cAf&Tsd?*i2Lj(tjxYhKRHpKxD1L#Kjn@J=a+j%~JlRsx9p>_z@F0k@uOO5bA(L zdd&KF^NuUixK>%IssBTd;eL~AtVxU5Wt)tVyrsf5B;fI>;_bsTgCtm7l}}{KPeHp# z%|&3@c~wCs9S9Lh!SmQ;fLpn+@Zz%%lRsmnOO<5aS5EHy$I3Sl_OWbY*l1A>JMS5h zMs<9iK}pHf=)+mogyGNE!oq>ZLH6%-UA;Wi!z1-ZL7DInrzvpKFM zGH5ZXZ1=vkv>ky5Wpw_ToyN?vLc}P#=*FpGA}gozZEWJ-&=!lTEmn@10`adVbn=Le zY{x`GwIqwpA8>!)DU3QWuSI2RJQEfypX;zK8|81JLV*9?`j-_cHrBynurU4N@6k-7Ircm;=UV=zvMZ$Ax3E4e!cxMzEd* zua^6qHBNL^`C;<2)!b{NXv#|}3FA(2z<=|mS@Ov7fyk7@%4#t5n9+U9lR@|$7CG+# zK+gH}UmuwcDO?ewSb0}rt~vSgG!>st;TiDXL=JpuKQ3sImiIRI9IV78C{Q?O z%N5cZ8Hse_FqXuS)!B`PY_Pmq^-(J}Itwm$#VOPy`Zh0=l$i76`yjqQWe>221=Qat z6@zU{BU)CWt4Oeq|K^dI<&?e`)D5{J4pWf?@(hh_48X;yuU52C(H>*zezkb7`y99^ z_sMYc2(EU!koxoG$-mt8XL7)U(6W1Rc`;O$y#~Q5O@KOM)7fr2#J9V$BK`u}GIlUN zl(f4^F1>(`Zb_NfsxCOS;2S8D;jbTSjy7z?w52i7DSz^fb=nBgy5E}LYBc0CVB?>F zsDWx_Vq@@jf_Ul>^l^9_oJ%w?#S3 zbJhWe0}+b~gll1cJOWerXXFE~{XCQ;HlBBNmBo7wq?p?e5y-nmwX~KEL0w{|Dn4h8 zDQ;)biQh+)UnDsagX=pa(|kyPxuSq3QT%e20#z@?lsw@JDl0&q`NNaU6G!Fa6?b0@ z+Pxp!hD^_x*!?@dS3=*%DORMNH!`ML%dc^50jR&|BfhPG8>@b(|B5t40p>^6#)XT| z7)Rhgui-7^H`qfe#s@57mIrESv6g(s8L;O&{5F|hkEZeEKc4=132atrwr$&ifnFvW{up!4|!b|w{hqnSk@wjyK z2GNLp*RI*=f3E!V_9t|>18UT=&{?rclpn>+W#RYy8!ffB$rL{cmAID71hN_@Ib}4O z7R;$-LtKAbs;>OB8a>Zgdoh}2BMdOmge;h&{)1#Fx>W;4nHh(Y^LRK{R=W2`OL;<7 zW;q_pfB#t6NqCp*5;ek>LRuy>k4(&Hivw~Ru)KWh6vmN*`YLXPUN0$RtcE*YZqwmE z-b}1}E2mgWF_-c*Ye&`RYpPqma9{kNxp1yIYz9@9mVf5k(ukWV{^)nfE{>mhQxANm zl}a_$YF4|u#-vH;dO{r-D#!ZS$>uKQKSa6X>cvn}3(|4z(4^o_u}-7%|9O~p!>pEO zj}pC{XrfGow%f%ie>Bp7{nEd>1ntkRi!07}0x88jURui$@FR~Wdh8Oo9omc#r)Tlq zJ^$I`CrISAeSPptv3enw% zIjV()nPLbA&*u6=*5zIIo8O#)Ia$Fonxf@64dLZ25(*t5@fH`S38*GxGi<=JS`2nq zk9wX%Vr8z0DQPDM(*vR5MOfb+C9$M+j*ev8sUfG3m7f)ndVreK>@$^Mc&i|PGEIU& zy&O^2_mlqtHB@{ry4A!mjTzGmXO-71)P>exnEoqGZ8UO`0_|BZ0@>y=8H_EH<4HDr z(?8-L7-&vOFiJZ#;LPRii@FF~q!}2F#&X5KKz{|;8GtNS@hcQJ!;qHaeBP0b%6oPH z!I077!^HDmj)B!|rvfFNjhi-CiHHUl!Zf7rnTA(+cuGmWzTL9llNm2l8l+{1tSJE$ zXdad#mzW;p5sc2$67KyhW|@$PGqEmPO8p0IX{~L`bUY8cJfQ_~>)3ggQT$fdN&wQG z!K?Bcz462nPbc5?fZ>kml|d6Baq*Fe8>l+=lCbqK)vB%4fVCRpfq;q~M{P;)12;d) zj?|`7z%ttLu7^ut+gNd=JD~aFVff9iAP9E!H_!l*dn~l_O*fy`C=p7Z7(^1Mv};@sg_W2K>k!WVq1m~q&P72! zw?}2gmHl*Hf8|J!rdkUfxMc34=z87{NEq~by3#DW%Ox$)1o=SNOgA$p z(m;B3{-}1bF{ZTez=?;DfSC;Vy(BvAY{*E3;i!&C`1i-tT@c+D{ER^xC7zXpPSe$* zE4YHWO*)sI05|8&DfE!EjX(DPyp@Qixj9Y^LgkmKNyW>1u#{C26X;1|`{QC`qC92d z=zK{#&H)Dr`#9L~bLq98CBr|d_-K7T7*IT{T%9d)93g80U5g{lW@EFX=l5u{khg%Z z3+#JCHN+5q3UZp=gozbde#MB5wJm!YJzjm zb=miuFV#-w8#H0k%a|V(BvOpZebn*BRx}^O4>L`{4nf6XYgYsvSz(G7xjaZ8XRH#W ze%yUXYUlL^KYcfE0h(t;&2o}yQ}?FyOD52ef7c0Gu6?Q-6ME*j_!w*zxHGyezPtT! z=YMni3b-V|lHr;`btDr5d{U=o*d1GWI3%q4KL6fejj|4CS@VAM;n+!?FiOH&)WoSn z=VkL6=Oy?R$H=<|*Y1rra#K{R*P^0B-$Py|(el*seaTu0wcy>!G{o0_&+I+sfpGhD zQa9h#Ho-NlcyJ}~jc-km_U#9bBVzWN$zKxT({mmO$1}W&w(ZLR=gDA7oXb#stu8!# zORKf0TkzAgxAj+7#Kk?-X$;CI3hY(CCBSlK`ug=Q;OOOu%Or0b$LD$D%feHvYuNGz zsrOn#z9LH2)1{X&y?pCKv;v|xMaVy9@M(-)cNqGu>G+j-*Jc|hALHcf!ilu8Atf^H zLO#Oxb5GTfK17)Q)gT$6m!CP6td`e1*lm>eaD1dQgC11V5|g@e3w87UcYdc>TbWg! z0%~Mk?qx3i;r#vbZ8IZ&uMqt>RXX-3byKEnyOQ)VYm06r6e)EVp865n& z9Uc*^KFmsFom#~gW`x0*i(&vq-w$y=b2Oa zd*yv9{*RxDZ;+DW${s0uW#PD-&mJ7Tk$fYuKP@u=VqZfbo{QU`7mI(M^ z+d6c@!|h?5J4kcog(R#tK}~~m={TFmC;%leggZf@`WAH~D%OGJ6ISPH8F+{QdT=7o zRWvYF%~|oTi(u|yl%|93Vrwl&nPOmtMl|medFfq)=DX5!JB`=rPuUfFoX;H1hWY&C z1ZLnJBNDW#5WyUNQPwgVfJ{LBw$ncqj?|})B>*t=%YpRnXj=LB!%3cdbp-ugLVcfMJ z&%0T3_sU)UZA^in6Zsv63ly#Qaq~YC6#8peT;jj^YBXtwjMG*N62+K@a1t)<+5&^z zZT4gqjeF2VrIkfA`zew0MRD@aPat1m`?;5(l}jomm(%fbBQv4rosc|JB*VI7U%+5! zeeFD4ecUP57-vPpqM#vA+`}AXwh2--AdxlvUOZ@z$o-_A7Gv~pwa88$+cJ=0D&oF* z#!^2BQ0c(|uB%c*!qQheS5K5I#exl|uKY@g(lV6&lA3rtYr;cP?5Fm@v!C4_5nJh# zy1$64=ShJ^z*z^Us*CadFmc(SZB^D_6~5QUwLg_({v@&~5JO(KPaq_IBXX3b|CC<( zsaS!+_5E4+)Ar8r6)?uN>{ii)1Mtny7%o@e1`7>Y0m-0TShYUp4a)Mnq7`dhPS%rW0cyJG|TCLm3$e!2CQPWU&jM`1B)p zU`3O*n*CS)#15a$<&OXmRER;zUi;l#A%XZj*N>)t(Kq>tJ)%3AI3zAT9=QD1?z2ef z95Ro~^LSyK0^c1e+#e-t?-oy?Mn^*3^k7!OpTgc^R;ByLA(8VT)Gt&;ps1hP2FFFH zq;AMs6XD4glF!cnO|V*h9B!oWTQA`reR*uC__73!67JC8{l*V4-{$BC!opE>NII*1KnK9ozvl3eLAfuE3t zEq_&Ux`%oNSH`gN!X~))ok!e7C1^h>oHg!ND{&4M&eI_u(oT8&i2?g^ugQ9;VJ<6Z z!PS-iTHv$6lB;9G6V~jZW?TJ%S5HB&db-{rX+Oc(8AAWQpLO+Sz6AAo`m+)snn~=3 zczW#cA8cwK7ZS$}69PdPW-Z?DJ*0tf&#Eay^(THOM&*>6`Dw56Z!NOs&hedNzat(a zl>6i~_qern4|x>-t2JMs%=?On@K~jCqhj;OIHssGnegdZH`w(FZ+QK|l2o0?5syuu zRqYf9r?5zQ<+YyI5cRuU(dpyAaIZuO&k~^KC=H^CQ(SA3z8P_jHHVVPwN$_^%&(Xj2a)von z)~qB^qSbh|I2R4)P3lZ+_A~_&7;N(v?-(US03xRW0bbd&kMn%nb*sMz>I7SO4 zOXe)+bjW>~Fj%{}A-;tXHpkj=+gPr*1_otF$>5vVG|qX^PsHJhKB1HOkY6~$s_Jup zZTe|@E)_*NCpZ$76$wInBfpD3)M(u8)WSwUt4r+~OB*Ri9vu0!t14ZilSnxl(lTHi z>7OQsCOe^*KxMYH+J1~k{kUZ7%Zs9jB}lJ40{6-dMNAy>$A&u`AFIgYZJ!_t1R!N; zA3z}&4Ypc1Hn#|M&W0~>d;5XX629~r9{)HD_NIl3qJLuw9)Ddns z<7D2CeI3POM~Z7lm-1&hN|EDbRu+{6h2|&@(17eX*iou0MGKVeUXM_6%A)o`my&bw zA6TZRi@cyM-Dp|f$0wb*;bdbD9@+X!U|J^ea|WYc8O|XHL^Vxuo%5GcnY_MR)#mmV z`@ii4e-gbkm7RjzJU>yIk5eppR7?01VZY~J981vJre5z+D8P3w>f_rtE2dDyLNoS+ zB=eBV)%bu>(r0p5I+j`1NwpT8aS`^Q|2T_dl~SAvt$B4?@h`nMuSU9#aG2u+cyU)> zm_vxt)hXYGk@7M}hz2G-aC~VF~pG>Hojl>7u!&tx>ANa@T zASCAFE&h2S9m=;6vgnbwPGta=c-_~=C_~Y2F7}#yjvMB-2NqnXY;OfVI~=<@&Y2-* zUK^zZSW*!;HW`&2SAp!UKW~H?xenbhR{bc~2=x+jwte!}h3N%$Oz*QeJpO|n8-fkr zp7SAMSpb!?Q&WfSECo*@CE>8liO#`@NeAwTZPpb+easeep{h03VB=F$ zoB|UOP&XD$lFaOjJb~l6mxJq_yi8TYQvcJJ`^y3Y%nfFh+HN;mJX;Mjd?9jV#^o3D9RB8KzoC@9>tNbs z^zakr%du*#QM50=j-B*m-v8RIE+osi=Si=%QH)n3@lr#LHTt&|3BoHV(cdOR_&(@| z?UYAWn$(lzrso<*o9uzYfKF!3o?vqnEa>G@yXJ1GpVXmZ(;pNl&35r@05xj$FS=0d zupgqOeeTTVuQ?rVT9bik4ocK<|BEhXNC=`1Yc#m@8g1~V$UP3_!7(k+lFQYK|4AtP zW4jAg`(Pt>|GVNDy7-vIzdZb7c@d2VlK=cH+pJFC+q$ONc3PQ22;E!zVOwQJ6SQ$k zvVN9Me3Y1j%ETmUVfCN9g5_u-#(K-<1y z^!K+t&?2I_1B%zlPg4~d?|kzjUj6msws-$Z@m5P{vOOGNs#-ihk&~b1P|N1@XJ~P& z|G99Q?Xe*vSJ^adg6->(m&kf`vggd_8OsZWf@k7mc{Q4eBs@EIusmVg^%Wgcp5vkZ z*CW~ahgb*F_z2CT$ev4;0Gi%S`ro}cg&f8C{zKhwLuhbfbkZb{Fmz8yc^8?I&96?Q zp)l9)9!c^2y0+DtRe~uLuCdc9A;OM@U;hG&DUge6Vo?0sMgaRUr8m+}YxrH->Ad9~Ab1s~MwLqIk*pUNYu772`eD(iGyAx=ty1oJ6 zt_&9`V?>6?lzAqUhz5ns5t1o$GLsCYOhrU8B}3*YnL<$%DKexGC4`7FPw^eM=Un%5 zd);$S-@0e5_gm|E_V(=aJOBOP`|Pu~`=wrBN!X3&_N9W-@w3A{yOcu%>;@v$F4OgR zy`7m$zuX+`KK048(D%59h;OSK>GDd6+>wdI+CZ)|cKP3yhrWI6GndUE5X?TpTaX_7 z_PWOzlci8=dG=c$)pV2#GWkQj`Dl(g>ZJX7?yV|*?{^^*WQv|28{FruE=ZB1npc-w zdAMTgZ^`}(?>H@bCAN}crB`RkzrVjm#7}ncB{($ZdDxysb~H?q#IKB3AzmZ>>&W>% z6TeW;Jup0KW=s1-TN>$|c9w6iCa~Q~`kYtdSepLJzhz0)yAiE z#W5)FH)M7F+Ze23D$ttrZs4wKqs@M1?}=WTKMk1*9EkZZb(wkK7yF$iB;RsObUrSK z{e@rqC!_HJR zR(LH85#QbN?7yc)Zn?NXT@Ywg29(Ho}vYQYky`HVbuskv98(nQI9Ti-ghfPAU}x zfjb}AH1~DHk;=QNX!40Qp$#8km&7T zIPPS_ZgkyoM|SzV#M^`5_~?L5J6(42!ON%u%1sS|gmnxWiU8%rALSjR6mRwfB_(l2i&b9ffuap!WWZ|l0 zWYJ3ZhW*UgPD@4i^RG?uT$+3Ky;w`jzUJC?o9wjqu8TUk+tXBO2JTE<^10Nujz7<(uPu7>r_&AkT;AwrR!fRak>e!TRc5!mW)^`{&i<&1v2R$G zE$4H2%YvTWNj{?yQa^g`=spqg_pHYzPUc&Ax?gVltmwWrDwnoh?#RG-E#C-uaX?Cp^t9kA>Ab_=uN>E@e8YJ+N%U>;6@{&(VRhD{!{F?W0?p4uW=!8M ztB}W^J743qVq`~1{bwMt7K^Xz_!no@k$lp6psCfEO}f*p-MZtiS%t!go&@E!cJgY0 zV`s{>aw4x3&w}%c96BqJ#Td3VUI;8jg zt8#-ju zPd3%1PTDK3&R(oo=pOI+Hc{au<;Uh#u)y!pe0#S1-0A$Hmj?0NwEQf5B*~6ljT(?TTW>%-D5?2+jO040<@-j>6SnP6TL8w~Y(L>2d{W%-U%rID9A zZ}0ecFM9iD{XY}!xARETy`&t=cNQYkZ^g*Fc{;ysej2|*X~6(Y=z8Sy#MRs#R4puL z&1%+Xr0IXRb?Ud3-?Zw~-$q2X3XBcWggNbcGVwe$irII_>Jp8>DLn@dlQdqw>*^1s z_bNZybE0$g1QX+4R2MAVr$7ymk;n{-%9IbUkR zks@DC?=}gG%0b!l;2g!n4m)nDFwF!t%Y^Ozx5$gVj0^Uk=AV|1G+s+td^Nm(?iFS% z7Ru1oE|}#wQgF{_fpl^B1#nmX_5y@Lo}bvCpA*S^KaRuIA?jX240=SwK(R!3_7-7G zzE^1Q`$w5nH6~v=b)EdSbk=!$m2~Hxa-C=F2q)Y^XrLQzkge*#c=_Z1iYqf-&i1p3 z)h}`>?}nELWCb*6Hf}HaN&aW{pmgLQ zhTC1#4M%?>i1M2$1kO(0Ssn}g>m+y~>p-Zcs$!#wORz?L`{Nt$_y=RJfl2+J z*9=#hmaVcunrW+q0=r8WkFCwLi*^5yPOK93ks4Pa<%oLRx_igh_MR)VV;R5yI@{Wu zr@_kjbzioHC<^(5q}w}SW8mm5)U4Fb48@!rlcSj@z~|c`6FOl$+r?s9zsO& zS;ahHDwcYwbH=mgcH@89N@;VvV{-Tf%KXEmBgRqjdd2Dwxd*Kioc|V9hL%MK+N;J2 z`^cIf+XP zLCGvJB6{IFPR9@dW&T`@rgPW?ZmDx4Pfv$^JFOUI^g&GMNJsg=yh<$U0 z{^D1QojYC9THe;EW`pmEKDej5>3FcjY<;Mmwd44qs`zcijcF?Ne;c8gDX14e%m}RV z**ktxt)7hk%uZwR=iopn#^grTEeUzbOg;I-ilX$ixh?_3bKp40i8^Q3363-R!Nu+M zJB|1hzoflZn9us_Z66w!pR6&oH!<(py-hKgMzF+SudEoDa=V)gXtq&{sC+r7WS|yN zcUUiLYE}sx29?anI18G^@#F@|zDmCw!xVJ(#N_{&T>3W7*I^zubJ9 zs$;WB*WZ$&j-vu}<~y=7+_w;lIbK)GjZ87h0K4B=KHkIkZxShQVU|)=x_clo_dN}; zu62PKm|g{56BV1x_2sI!8Y#{0HqLSu5B6epkM1ur{M#h1bRN&vgQtDDs2;cT(zplg zk4PD>((?JQXZ#^m6?!L3xo_TO?)nEG0a414FU>55U{EQT*{8N6`V-f@a?p00*Rm-G zI*Iva$7cTa`eq`pZm6iXLdd(^Z3_hLKGtf8s!yz=$ezh|pgXZA|&h@bPYYoNYkac;-z9HD{F6Bn}})Jrd}(O=a>_jq<4 zy{W0ps9UrkBce9bw^A<-^eU)X+)$km#PLb_t^R?EW+9q~e~vAmKk~P`-{jz(FG_L6 z(03b6aL~KJ^FqB-Q)6$z`|qBupQoc0URT9KpuE>8|190(s4J+)<@dN;!B*SqD}ovC z?!9Gx6go6?pF3xyPcQK^@a?w4P{n5thq3|U=1?Yyeu=gs}QbhXfLDZ+1cD5BqVbaxkd$Gz#lXJ&72DlWeE6Zx+U@UR@e=)o_nz`x4@T>OyB@Dw_0JoXPX2!NyVRA@=HMda z>gkI9f7q>@E;yLEvb$SeaItZ6+{Z3Ko~+xL!eSyz_IvxYp&kSNR zb0;@P*R>K5_KCVxBk|D}v$b^$rs;@R2`J8AuyHWE;3Z^Z;YoEfqK+|}?#J1%u)>Z& z*P3eE(_3W|Bds0H>23{OlA@vk8cer3DCOlzp8ARm&RQt#;Q$9yA3;1yLA9Ei7hfCGM|Z{a1Nl5J&XuX-WN^G#M2-*mlBTF}6g zvm=2wF7g#}G289o88@dTc4~81^v;G9!o5mj8J(sAMDe&7Ed3M(LRDy2!asFXo87Mh+z9PmE z&aEd~QsUzYIy)Q5!JaGN8dlG>U%-SClmp?(`R`AKY)4jdT6aEFGd}X(do^7uy(_Tn zB;K%8p%Pafdab7}aU-?wFS`!oh#Ro>@=3mrK$_MrvphHKy_z-Pd^yvD zf)Tcz*6UKcKs&gfI`9@@=;=4}Ct%%F4}_je7U;QvYoi|`@}1||05@xy4}_izUCJLp z^KKJ^$4fb8wFRRVm>sFO@WmuA`PT@YyoYu516djvAt zIcBs{!I4lpfMY**XAT2wMCMmdUj-3CcmUC@hrvj$o&Z6*Y1f522uVwr0>6!j-0m0i zLJvY<87M+aK0lwfL=)+YpQIML+~d*qBH_RExh4!N2$h~#hy(@UZUe9y$bOzod->$D z={HRz#YflPOf!Pzhx-e~*(0t^7W&SF?`(&yYVh(y%N_dYSn3%YXTRSO>NB5Q5y;@y zZF6y*x`_!#x63M}pB;ql=YA*u4bTtnAOyA@hMp;)3lsjwAS5OtYr1xA^z&k$MjG@W zv|b8Ikle!$S_*0}2fMV?>K=8m4DAF3^>J(=+8BhYKQYIGgm4F;^_n0f%Gc}csz833 zwwNsTRhB}0g@uSgD*Ri{8*5V1PhY&kMdRY|LZoI@AX{!&He7D+F+%9e^%Q~pyj7G# zZ%DS7aVM|n?sH7Gun~2HRHJ(`+z;S->tQge0D=(1KL#OD5mVr|5z(7qwnrc$e%zVR ze`TNu@k>-*^f)*xRxEs=?DPj>dDHiMYOAZO5w(=3w*uqM|KE>oxu%u|62jdG;NJ=v z(3K<-65#6}J8#{q*TyIEU%&!XW-_05K_@5P+Y2~am&f`TUV!#rz84WlqIdR@gL7t| zH0copGUqavYtJDPEoXbHCHy3l8AVTlsI)bN@^dF?2m(HIVg)_yoF8 z*1e5?64DP`qRk*4R=cDQ6gCvvVz&_%bua2H;BCVols`QALE{JC<57T zr+nn?M?O8dDCq_BHXsLqI|ZcuhV3|XJz(2m=y3t=hxMjsx->9qqgF%*w9g=VN=CvuDnHA^3XL8#|G9Y#2#V&=E^%|2hO_eN*!A5j!*Wr2)5rpuKZaoY}xz_&v-v7!-Le^9i zxHg6%hIDfWFbqLpgf8U|qs2q1jU{2y&D^fVZ?ehLK7xTL?lsn2klMpTd+<<%t_ZRs z((eg74^ose7)K>|35?%9087zyd%&GW5_i_uHMPCL2g4KbQq=zLG?B26uQo|pejWar?23t|(s(~6FiqNfxp(p{kC))p-2}uBmungU8cU1=s z@#79?|CNCf)aczjQ}(#7apIW9=apIYrBU>3;wvmcr^(BXfPx@AN&i&?*-usI&(A%E zKILpS^;@pJrILo_NB%BTS+Yxvjr4VdY-)aD6kdLIR1kz)^j3Wp8y+Wp{?b$$f!uYz z-&AvCCZCO@#eiglq6xO2Hw8tVpdZ}BG2kAfZZn3Sesh0<{r?z)#DKY`wQFMx;s|5R zg5H&_m+~hWYLsv`-kY-g(yO?}eIeU)KYC7x9=j{6`9iA$62je;t=9w@(cJr?n|U=a zmW0fC2qm^;alu0L?eLE?yWf4)GQL>z`;NG1H(rQzpE3Ei5`9!6(2T8BPd#LgKpr?- zJAXt+hFB!{sA<&l!%EnQ(ha?YKt#A7!1dO{U{vv(0HL3<+3O74`7|)s1pGE4QWj(J z0TJO2LSPvvLi|flE%mdE$LN3aOcaa=Cy%#+!?B0h!?D(HT-6{U+>HR%0~rua&Lso; zgIhe)Or;6N&r05f1&G+X;@3ejhYQJKWc4vpoV|Df+J9&)`nP{>*W(JGF;mwuTe+>h(f@%5m3Q0Q3wbK zcP6w}!kD(`L@g?MqNF~FRqC~$3L zgw%h%9Xg}668@BeVno~N*RR(ms&LrtJq24+ksN~r_<~~}c4tOUdes~{qxG5~Ga5ON zy?0NkCp*h^iq}Kc-}7NH3XmFW8xoXA{(j;6FQ>QS_INS+)f1BNfJ^prH>KhjPZXCT z0_l)8X2E12_UzZkQttt3@;TUy8hYhrpfg%;Jq$)hK#n^4kBr15M1eRWipA(iJAW8- zM(d@Z7>z#rKy&wko5t@G(X;*UCtneRZ(92}v8Sj>WmsyUGg_|+GNa2%H|SsamOVWw zPg%)(FjfQ>Bb(S2hU0y=9}zbYv)!>)rooF5BmLs}ZyW}<<6cY0=Sg|AA&@K?5~tcm z?U%j{#hsbH=&}zsqu-Gslh7Hhw;l$gC18r0ZPS4n?r0pfjE+9Y|CIB+kap+A#S((J zYAaZb1k!)nw=>h-Ts`f6*YFfI1zwEA3Qmm*)z(svNt|bM89maAKn}({f8|G6q3w4z zI*!)N2>xm_WKfj|K8(QD!(>!LfH<&eG{T))OGuakzl~j5E;mmi^zl7d#-9!?-;Bnq zde6$t58Vep&h8HQ4vz0rUSN&yk5(!Yf`V{&>|iyJ{nURdxhc+1mi{)hEZX%7e=jUQ zO}xriX99;#Og&k=xBp2&6JCD4b}#J9<1YMMGjxSw>#e$@2;^CD>nq<)p3(%|bQICn zYBqu$h8Ax5p~DdF%o=Pv3_WeYnh%yuGi%&>WW-E?-$p$%N1S}1>jBF^>5(%@?Mbv{ zu==of>7ruI<6;vqEoqJ|=7>$|{<^o+3dWa^ra zKCoc7-0vCRC^kBcz)MdYyOW$=?c-v<`@rUWkIfem$i+jSTyvK6h1Gr@xcWqiQWv(K z5v_xd!RUiq57>4XdMM8N`?GJV2X_}HDh{mT0#C)c@4Z0q%8pwJ=>FDxPgI(`v72GruKheq?-6$8f9vdraV(5Cn zw!_e)cE;bI=pXfn0eLz0AS~T)tN?oO0n0$?2{Rq{UTNuNTb(jBib;-7t?K}Tj~I6F zk+>$R4xJBJ4`edOQ!s0`{J)o~O?+A(DPKhfDG7lQ?;&V~) zyma$(KBbs}nuDg#J1Y=KK~qgHv)D^ce)h-NkLSF+2b+&|V9r(Oe89HD;1dlb&^tHf z!;7=u5fhaHUXWpjpM8iWT`>GWn3_hHf+D27(pBK_BcC?9`RuB1Pwj&#FaVvujFm(i z`WEwhj}Ll4JY^%H(fEFO}hPd@y-Y8I9u^AYKwhU;ab-a30f-e+$o!pqRe zsZbBTzVhG3CGCu{ifMz;Aqa}lrJxigOtmaErrix{%c=5M?sRSLEj=0>5D>5> zn7K(*apMb)JM>Lgh5Yun%8^mrk>G{IuDS;GhZTe6v{*cLn(p<54w zksL5HJ+vt!+=;X(u#O9R*_qqPRcSC3@#73=U>PVvK9(Kr>9h}T=~l~jrPSnI)QhIt zvSkY?l{8WeOHj*B?!%xU+>a7i4P-w#ip^INH3h~mrqVePo7HZE<;N&TNK$lW?j@m> z>7kWh&mZCCM|qHRKP%(IP70w6qu&DdzzZ^=A)RD`SXOPP<84g_!$&+}cV!dr2kAgR zxbtYR?J)GvTKE$j{MW8bT-sC&xHbl%c@0Hz&<}(lgf8WeARb=r?o?S{w?{iiiAv`W zl!IN_&seOk>=c*gMNklgt_)ofWJ4@2KW>L=aA|(^=@>FhrhN;0%&xaaM6Jq%r@zluXohVF*ep5@G=%&4 zTf5PJZZ?J@pGE?N)4#sf5RnABGD$2&Dn~2J@KA&<<&PrQQ>jWz`e!crHv7Hm%?`*0 z6`jXcq!GR#frlb=MUWK*2AWEpk+~#VZ%8h7nu@F%mLfAN;)I!35j`%3Z^*NcRNuf$ z(Uw{HdP7-14rZ~vUyM%s-a;TFUQrEBm?#aYuO@ZIWNF=pqiwu|!;qJ-6GEjo}trIgnKVO>s@s8w&2APjl8Tgw1@;>ZY z#c7cqc90P6M+vM4GN8)zjPE{q;srHr68qkSY}bYbh)z(ODk&y;tY6iiY)xifmzO=1Kd3FwO11EKGKfP|U*yM3dGu0|LU`m4R)C!AINL zpTKR?t_(LHNm*0iw~>!B%j$FJe84hLe8Nt>xKuUT*;MI9sDJsnzOYFzIH>Qy9@O79 z7&V5@2doD&ABkA8v-Q*afzf+J$pRku*2#|?(@atu=%jte!dHx57>4Xe2Rb%IP5m%!;3oy zmJu}-1+I;N3Ze?!p!Z>GCHyG`MabodW=_-R5`EnizH<$OMBB^3mG>X9SKePLrey#H z;m(BCD}rn&V5uZ?ywAQnoW!=AfXyQiRtS0)`DVJ31!k& zEQ&|cAnXGLqJVitMor|BuJa9dSCJd@KkEu$8@m0CIs-I>JAALV9)_ZB;GP(5+J`|< z1gsGf{mVKb9p`UBcql@bf>P8!r6Bmdsw{upH9z0da4?ev%!J+nz3zt35IY?h`|(hO zt_ZRsEn}qpO{Q~CA7*_Z`t2+hiR-_6R0dp#}0P=fURiCRm>R=Md;SUP&5jxeKpxM7UAy7fUk>y z-^NIk&q8_>dRGRPffA&aJSFVbA~fQ6?T}EiCujV7bXV4j-Id)Fl`sMc;eM3BdLRRu z+#fx9#lJo4f_!W*34a9vEI{9hrKqNTyyt?+8?#fJ?p?$S&`RNC?Y5Hhy%aQeLkun7 zwjz+n;;FeVDt{{~iwQe*j*%<_HXsU@STrEqnGo1^7<{6EuFPgrJ`jSCDDV*(aBTz> zkbM6I2nhF}eyxN*rJx8U%-P=UZSF3r$Y;&IcD&+gcqTZr_qk}`x&(G(*B@AmV@ zAdq+N^E&Oz%j>u)e(88vq*M!RM*gjnsn8j%w;l$gB_Jg--83b_-I+;2ZIQtg8KQ>FQO(UU{^c6(Wu9sc_&n#2Knz3cm}wZ}n1xH~hj9>{=J zg9mCp-13R1I8ga4n35abhxG~d4Ix$^&zWZxPO2QAQk}sIkfp)zTXttIvJwT(*ku1& z0rvGsTKKqiF;|G1;%jA5QtjF1*N3JbW zJ!wn#(Rrql(h*qU$F%a=TmT8eGn{}o;Vb7`GQ9L~=lZVF(^qek%+g(&s&DZ?AWM5+ zE$f_idi~K~KjyHkoH1-Y=V;FjfZ+#1D)irN#?TXO=}&O#AN2rh?f6U)dp@p;C;s6BYwu8V9Ta_AY{_Az%QcKp6I+!Y5_^xZY>q=E_`)P2wv>6+A3Z(A)>I_H-LMDx_2{Ix%;gG=sy5}{& zCokW;_lVj87NSpaTRRtn?Bv}g;yAAj9h1Tfk!jek(>(VbKOLv8t`TEOEFIH}H;rY14P~5EzL|0c)nQmwKEex>5$l zAqb4nrTpo~Ix;@+j!XQWTk(JEB9*^7p9>Di_F;d^s-BY9f(IjXO^_Mw2;~CeRfmaRqu7Nd+ELKLy$>JPi_&6oxIkDb7a(O`Mn_uJZ>nG*qCy}aRzs|X~ApXQIe zxfI{Xf{zWuvwX*3GYZHIQwF;-+-bnto&IyTF&K>k%NDn78jWzLLcmVvz;9zLDhb*m z4?PtE%RmvLaMGn~<$t4XkofdV$}uxrMR2JHqp0G>A^o&FzcMHYcN>7!K=xxW7nR_C zZik*{{h@aOCuG@S`B9fY8yb69>ISF49$z;47Dl}M?60mS_2wxvxx_F-`MmHZAp-f+ z?%`hGjcrcS>k9vTY|G;mfo|DIvObmVwgaV$$zo^(}R#GdwqbQD-z_7X0q;@?PwIO!81zD0DtxJ&^erYurySF!vp@ zcb7GHQGJjHi_g*l=TY_)Wtz%9_GlL34EzT2OQXcsXjWTU74M%7w)jG&1q|d>ZY~k| zK02o5!ttakt;`WNpRECE=&ufO^8woqgO4_lN$YOP2X`MPCIT#)2Cj{Oj6JR-fq-!L zVe6%!2$_eQoT`{ zBHU33ECWS||A`hazxd#Z#Tw%UJB_&2CG?gaE7-3#%$EqCfrM~30$2}ZKpzRNZ6WO4 zu_91l!1?3J(+OCB#ys*SD@t6KEfBn%nFB$m@d6a2GnFI0BUJx*#AQkyezhM6 z*+nJ09a7#eZJyFa9)b<1ojj@%1cW;Zfo+GuX9`&4;rFjmNCJ2@23#8f-Lo%v00M## zh0vv-2#tO{cITF@dw}fW@qzXO`j*=z!4K#?u*dX+VofhWLJ*=5x+chojA#qRNX27r zYu)b==qpVQfQ9Hfp$f;cxW%?4PkOO7EsZw35KWw*{!#20$1cbbX%~Bbgb0D$#%KLJ zX^5Ww8*NBiPrpbHY(&XFJZAfotc=psTgo=WK==F zstV@P5E!9LK{2u;|8hN7I$6?1S6{{Z>J1fBFq{4az~Atiz9_${4i81>iXbaGb;80S z)$zN;%nN@m0(FK(Sc*>PJR4n7aZfODX{ED>=I6yr(USuA++QoiRride?rmA2IEX+p z4-4|9FmK68s!^&bh`K5QTakuCJ^I)U2#V0HhoPvVnE)~OkBUTrPpsFjjf#j~e{Y2z zjMhs*DKe5?>a=ZfvM#Z>G7&$RU0z=f4(m%qu?A%tlMC|D8Lig@nNil|!#&Q5InN1n zS_Hb)y5LPQs?^P9`jE7bUi1E5>^0nY7%xV?>4wddY;XM2o_j5+H&eSIkdlYT$u8zk zE~Y$<^dhvU&4kV9s;=JJ5nQOVqi{1?Z#@h~-7g6c-J7OFxPy_XB(P=(xHdAH_>^Ca z2P1STC`RVT79LJCC(++XYB|7g+JbiA9JqDx2zFZ3|AEUC4@T&kAT#pfP4Zl!oTR$1 zvfsaOUl;;bG}?jK>J~UPyTI{YoQXfFI2A8Oj1<1f$3B&i>ns~z()149i$LCZbiU|~ z{=59cA{w$Y4TR@mGfM60L7$-jff2g(Fc>8OeOlYUMk8QdttfD9WR$Y_`Z)AHZLNeq zrJxvT&-N6SbLHffXwGyDhU<~u0~at31F!Zr^l1dbvWa*oLRSP?(ca5F<5PQtK1T9a z64aIE!~3!|%5pWn?@~_ST@5Q&?m6)zcqw8|&^;m1RNomL)?Ax-m-P{_FGHHf)B)XX zY(|T|=387v9>P`>N%sc*x(tFMbn9U#8nyEGcl^f$TXbzxS=6qw|NoplBkCt3cmyMK zDJVtV*!$z_l?Lg*m$=;K2x83W|}7 z_VxvzXFoFI{U*y#P<-C+iQWJ<0ecBrN-13v9?=M06J$ma)m>NbTykpQws0A-(Yx^( z79$atZi~nkRyCq$`_dmMhc4j7D52pVPpyLNNa)!HlhIx1+TN`)} z-J3Rn&1m}S1bWI5LNr3R9tNZCHw1`;f9=y`foKF=8yQtwCoAF6r=d&v)2E3S$3Cf< z5$g|M-4arN)NwbsoccWWc2!>u7%lNogsup(BI97>_^_&#w zCOx-JZ^CGc`7gB&yc8W*&Rv|8XXZU)cFuf;`6(F!S?#>CuOLe}PoS}rMKgl=8f--u zB$;>P5sc8ShoMLg*aP<3rU^FgIY(fdec-pzkSF)d888&_1EYY=PmTr4KnarT%A1KP zVkg(qi==%ZA>5A=SPx`C&fm6G3}kpIQ{P{luISU;3JVaa z=7=gQr>oVu%eVL&Ttvq40+jOmeFN{c*SaT8RAp${R3Q<_GUk~$>H6hTH$sgGT04IM z*nn!^miXQa>zA7NQ4D=`+8T1J@}Q^vnT!Jrymqt z-UAzv!6|+`_QPLqJq$((z--afzxHLKz)_pPwULqf>pO~gFhZAtVq|9|J>vV~{#f+q z=lw%Yr4nClz+t`;_SCg&pqmgFiJ)ZG;NJ_Gk@|}rSGk%Ux_-IF(q!>pu!hA5kv%5f zIlAQ$?Ps;$3NM}SN&)vpg#Bbs6ue512!rhrkNtgn^jfgruS8YH2ZILYF?{ z8i54u$%MvBdJfdo=;hR;*o1Uv5wsv7+>HR%0~wI@)*HuT^C~5Fq*(^l%Mez;0`$|# zxGSdM8C4Zon^sOg6@DAW)Rhn1dv|tJFELhUi%80T;04+;?f29loiSFT%!YxfEem9@ zH*8v(UWub|K6fY6ztOF+$sRD4$+7|mYewNTRCXf(>D1@#FGNQ%oulk`5_s!?+rp+Q1?2f`h zHR%0~wH#|EnL}3eNer^!BzlmomaT@weV7 zgYk`DK9S!2zQ!Y*RjYUb>ewDwyFIRaN!KH;Im&DaI34JX+GrkgJH0rauHiOU-ZU!M zQD~@SQ5pnANGhW{hC@nbUt7`konjw*VpZ6K2{*r{(|O;nHx8(_+x$2M!SMvH^y;i z%2Hp)2mD54PDeVv1$e98W+jlUrSG<$0KSkW&`|lr<-Je;NNcH3PU;!hd=diYuR`Yo zwjBl^3TuD=^PBR)-HC|-OC;B>jexR=M7=>kxI3}+Qc#4#BKf~`ezJdP9s7#>ZA`a$ z0eTgPFm@hvo_M+fB!s&YTdxT+qKgPpf_sJ|=_Df_wf9GQNMRx3^inyd?Bge|&wQcx z;@+ZaykgK9z9K`Z+-xsoyuhQvt?7;k?%S}z5~$Xvl~rul}2+fDbYw}YB? zFC7OzaJ?6UwfWhlREIL?iq20b5)}iM`vccTMd6f&^U#CQdMPMH{LQ{)(>?uBE_2p%0_9Kn?-zmT z(0A-~h$EW_y{Zv+Fj}t(G9#0Y4tfbgsnPoaCtNwT3g9h_wKQ$Dr8!Zx?Hft#DfR=> z_zlbKN8}vrnzKEl&qy`x8gLRrAQ2Cn?0fj03sj3g&)iq%)(M-D%JrZeut&okjMiHZ zgHf)zzyH2X8R5>TfsKRyvilHWi$8h|2tUrj9asj6P$PX`nXZIib9%oMePG5z5+j-SB3~u~xDV z`RaJZp(CD8RV+^*dKlJc7^-Jqw*!I9>$rb8E%TR}^yPgqF_F!LumR2Yk>WFiKaPok|R2W zn+h*P`l^pexmr*1e!lo=QQ-GiIUovkJ^vcs9% z+Zc?x8we0`|2Ro4CI;;J0bCn9v&!x9tk7S$u9t#hUnX#4cOU+cvBLAbklo3y4cyKpd}>TGjm8&6}F-ycj@uiIcB}}FckR! z$J}lIXGOre?Z50`C110B26`}BE8$NmC`F@Br#t19{6;D-9X7kpzY<^#Cf0qT*uf|$ z;8rU>jKG>8Gn%Rk__m!O;-M~8q;(r18@#n>mSVyQjxA62bXN*JRd_&k6fZ_ZLCCn5 zpJ#XU=N3A1Wee5-TbqkZC(|TE?02^$57;tw2L2BDdR3A6V2_478(6%rbVlnn zL1x6ibXxqzwh!kk(?b33im>Uzibgc=&6IPCSUYpx9DQ*ZaYG(2MzJ(p)ag3kSFe$k z9go%D25g3wNY41vjfJYd>4D&*tZ?B9*wILHp9CdzM(eGI!N>^s#LhLjYhZ!Xws6Vz;7d>I^o+~ zU}D6NbASexfg+T!RC3DR=U|D}lh}d4`;t>{zz(e@0V^%yQ#tS&6ofk>gVjLxKTZ#Du{Iv{ueVQh8qwLLghI z@}Dg7&WxyAeJ6O|l6C|(qDh2WB!~$21GwIL7>t&HzKrc3`!Z2!8DL#C_E-1<2ODld zXS80*pFmW2*32QXontqt)JR+Qo4GcSkr^#Eqg@|uc0p&fUK37o(r|#1>QyDD`zsSV>s7H3G3H{EpSR=bzrG zoY@(<`&?G#JJ^ht7zDMTGg@yw3`X6+ir9pIWh4fCat2%*qtVubV*X%n2I0*Ix)c$kG)l@bfIO-oq%We8IVnQDLOfFTZ+okFy%(~L6xsxf`P=Ecubj!{hE%u!Xldh z!%5D|uodxLEJyDrgF6_l-RM6z8$(epFz0yeU;8vMF<^j>{r#)dp)ZEegV9 znn)P_n-9fhay96|Xub6?7*znfqs?s^jBqE`z^-e+Z(}G*|JgutvbE) zr{6t?C8=lgMncwaHNcaDUN~dFJJV-TN1x}4JF#A`2{IyswvScyJXK8ZLb^sr$c`4m zLbNyaWm5Fn8DjUHrKbH2=eY0+LWB`_+gityI=fz9V!NoGG6an7>jP^)D^|GOvnDVK z2#k`1-Io!3OGUpu!~FoRw;l!~Ibd}EYSYm@FU|lW27ElSc5P(zLD#tnI-~VcP>i~N zeh?q%oeQ0Gs922utz>!#{OsB=;v`WFEJT!# zmwj4R*c6x3Ws@a_G6wMqLg)Q|(=n4Vk=-kFO~2j~3+zioEWTj#V3w_tgirFuPyJCZ z*g=S#WCD*#qV@ZP!AK2QN+j{GjAVh;0l>A9k@}-_T09t`OF=QRYxkU4-u-#;k=B;& z`Ax)3ndotu2==(l_49j8JQ$&Cg3O5gC(D&p`7;L%KSn>U*ZQ>pi;=Mw$=1OZ)t&SK zrE^sekJI7BXw`%}mAp;=Wl|$qq)?3iJOar*Zu`idkp9qzore~Te%Ot{X5^dRiO+V! z=+?tvBnP~Hy|!szhC3w^1rGXLyEZb)phlSB!3bRnic!MrmG@&OKc)t{4owNFNk83& zUX^wnJ0%+9ndirY5xOSGjQD1bYL0(vHgZBVcD$3KE{Da)cK?l*(Xlc~ZbWkIRyW;1 zycp@K*moseP5khKEbGn*ZEjHnGVsi*Lo4adopR!%yDJMha$qw`bm4jhc4@fNf%UtM z!Ke|KB|84E(Fpj4L=?C-GP+sP>Wc>>bSWrC{2mfdqv97ue|0ga{TlXos|Kb;W;EE* z=FDEYSOYLSTe$Jq$((z?60L zzs?d#n*OCr+o^De6GX(1bC3rVfGPt;$fsjfV(D2@SNd@2S33pS*a2{GrYw(5X!$#j z8Au3sBY^cl22@qoM^U(|(!@w_I)ga%CA?#VY&(v!gu5lejAWya5$q%|JY^$-iONO}rB_-@QWJbJ=8;nD^Q){s8F!+R7`}=$T zYZQ_KUYr5f#wavBmvaXM1mTSbx)cNrT#QVgyeSn9)pY`wSkq&t)`!#{ zm4k#Jq}J$~AR|)olI0sG9zM9gz2;5QJ$HIoQK+;{|F=oem!><#fhtooJ;(7vBzUW~ zMMBf#g}|&^b@#99HxNkCT@mJt?aORslP4NOx2uH0MwIjPIUY;W*6#G5yN$ufsD=Q+ z|Bq2fTG|x&ZAA22zL|$%RmvDO0qARcB?IYLE1SOmuG!6Iv=0_R2e8fVIlNZ3k!nhdnDdkes2(d#tlwXU*^N^#Pm+S%!1Ab ztOqh5A(k*!`q(t7mL(Tiopin`SbXgEpR`0&>g3NShds8D&||`jPv)U7EY_z&DSsS! zD}9<2Zuau;@u@vz*vy|?7(9=!DIKC~U_;l1CNi7M-oy^{P+ z1k!4V<$erH$d_+_Tpj70f(2*a&OF-%LS@rma#=jW!K<(Ef^+pLE2Zab93N|l*y_{Lcsm4g zo;BjI2Mv#5h_=b%gIiqWWOz6rx zChM75q>$s2nfZ+CZ+^of)HR{NI+*UacWWBwPH{Ec?|2cCmQQ%!*-OV(;A8vDFQ3XE zfjs*B(`q97;n(-=?P}-_gx`QoXm4oLE-?1sP8!$l^q;$p!6*S(H&L`{<_v+63@{A@ zTpJlt4@=>(XTw?ve@a0yk~9AK>y=CCx7v7$na_>|iJ!W`?70H_GwC6p-~ccRL12Wg z2{NO)qi3cNe>tQhJ1{hUr$_RG`EiG;e{5B%e z`BfAP#v*>45e+N@MW{_z@a?4+9MN-L4tWDPKf<<4|9AMy<-dI6%B@rt1`@*kEP?ev z1{Ba!M<&Oy(9Y6Jzd$+H{RI}F)|lYD%I^biT}e@y6tz_N9oq5om2-QwM(=^-mD9xys3b5l)sjWTw&<@;50N8dIe5QcTOk>l|3^yOtT0<;8 z3CE65LFWUOf#PH2^?SfJ%xUoWC*uI4Idye0@SFt_QS3ZumLv89bUt7`koo9rjj`Vo z$Zc6b{k(;hG*<)`AA7Qk94FH5zW8pF7qqP>;u~J^r>7a+;>{})JuY7=GgRrAd!;HWyJox-q z28xgQZ>#S=Dr|MBUtM#t$@aVJgN{Bj*yDHaGv^-S!RNnvAoB^^D!p)WmEt!i=kJHR za^x?;;xjCzVyZyL_i44k)nO|KZz*1Ubh3gjyl`?JcKTFv^$p$AVBnCBaP3@^prG7^ zS@Ri|)H{b^N1yOw<^VkS{MUAve1L6voj2tJAqNrx#x}sUvHNN@zRn8*f{;q1OFJC~yhcWI%qvhVamNyb#fp1gjV5sfX+D(d~F|8nEM#&c8Ae1quePjf|RS4&26r5xNu0$QbW#&m$d(u>jXq#m|vn z^W69{&Tl67&QW~#4c{i2zM?7jP!utMnu(Pw$mUYew?W^SO$uaiyTu? z=x6J;inrplQerHp2GNA9un9SZ+2b=A{omgT84!nik&|8-OXkf*7HKvG(mq&#bRDAJ zCa^`XocujQLnTj*-&WL?#ZgTJS$-KgjH^*xr=)9vb!H*=4c~Fe`RX~!9~;lSl>{45 zmO>=@!+PAg5O|L;_=p3!kocy2aCc^65)#0QX>35+e)>!xAl#kVdMPME35E|C*uQ)% z`5M_A@taMEO$7Z$RT!I4YR9`i=uv3BCdh~gGkhj;N;CUy3PuN*3+H%XA$q>>X5ZwO zs-9gRUpE;2Vt$QRXND;3$vYV{^=q->xX1M5{3QgkLyaepN?KGbsWt zBGq6u}ISh*t!dO(C>i|!FY!p(aSNZadWq;nnu-@~QJ&a3k}!9Q1e@vKe} z9*oefhrvh=I9*}rUl%J%Nds^Eu^H8*wdsH_$8blZwG#f6f?^crowstY!%vCv+H*nW zdx88-;Onu%M6AX4NBv9ALRYk25oAT*chj5yii*+g_})k_a|6i;OVLhxr|Yu;WLH`e zFTV{KdDwtgFdC8Pve;Uz4b5@83A7bHU6U_G2m0#wQFNAk}zp0gsy146qKTV$}shZeKtOpABJQsyQj&1 zfQruF#!|FRm@WjmqV5-c2 z!Ansv!@=zxBs59XA6IuTKSBs2kh%P-qzRc{#B9GxA2~?n7Yj3oTLLQJh)8 zu@D=+YTX-CLj|4DdQFfSrOj2{G2yE4ZKg{R;;e-Ab2sfkk*27?w zd&b|N{9hAn;BZEFMj~CklJL@)ioW@9TiYh>6v*{(%(b{UXt15dw;{xA2Na zQl?y)5f^V0th&;$sjiZrL?BD_`sC|x{cKVfbt$*0Eu(-Pjh=WeqtAQ6&1k*#Fc?(; z$4h_MG{MF_%$Jcc1%4Z2kw=Yq1lXhT;~eILWuOSjHKu-fnKI;e+(cuHLVxUX64;+n z0mFd}fyl{#`4K1xcN>7!K=u=L^yQOJCch`Wnr#aZKkMLq89mf{GWV0H#GV_>fg#i$ zH!|_^W8f@k^m9=5RK|^+rGY=LN+Xcloe2iVOhRXc?IdOPt#Vnw_7f`h41Ibc?omG2 zb{KjnfP-GB{xJxN0_PB|T^s!zVAwGSJqWFrf)ZqY=F=%g3E#nrZjH~5UbCwNYT)op zzznM|>wRB@$59LGH9*}tRTew(uuWsn~GfUq5LhYJV_RK zAv)Lnj;6v_6k$l}P_`$I2$(8T%Ty=TCM@$kz*!$}E#U?qk)WprK3m10?=}V_BjB-C z`^O+8DQyb;HX>ROGDU9`gF6U;WuOR!b^ZML`h!o&;YNW=S2IY}d2PY#5&3w88>3!I~JWZq}V8K&WGr=4y`L?&izO1BSO; zztHU*bK?tt#j6P{U6k@U&Z^7gE%S+9O83Dc{G59K5kvHW(zv4#c#kmn6j}HasBIdB zaQ9(AGX0l)I+~(K4~#*$`G94h`1rhBoLy|OJMxbC3tTf1QIQ9kf zP7JIEG9QBct-H7;9Bcg|_>Ff_&n$;y9uaq!y6j4jKA?f;Q>-tkoZe;jvHx-AuLqex|B?_Eifgvczi zcVy3!B85~WrAXPUkj#>Z5E7!utb~y4mEUo7j{Cd!`*4OH_w#u4*ZX~Z&hxz9@6URF z#OW~yv4>QrQ5^mO`;G`7S|B29Tp~U~{CdETRDCWb1-wR{0)aLN_*AS)|N83rZv~}^ zD2f}2`z~z+Q=qfJR2E4Jv>~4+3OPNX8^rX;vy8HOGrdh>*z5K@W?S_dO!Rb@2I&Sd z7O&NKGkIBN$C(eP=t+36{(Hx3KbI?K*Wsj|TBhRgLF@fYpBD#SrCHCNE>d1%hMAr- ztkKor%J&G2xh4Hg@9cj``h#@*MF{m^0_6)`P!LzpNyNj{i zyD9vot=-CH@1EH!a`du(B0i`fN-p_3Zp(Vo;l4{A&*02=U=Q_#6VuMar6 z3aa2RL-hKcHj25$h2A5=$i0XHXR~Muga9Kxpu!N)NM&flu=95m7(rGD7}+0dI&ZFb zwQTVHgb7o?&QNXe{7;c8c^WjmH$xT$Mvy1O80lwA^B4=waLhgvQT1s{5yYfL+yC5- z9xjOpIg@m{*oE%qD^wT-Tz**j-sGfa+s`{YqUHF%;PB0XF5&hKRioAKW{ynGyq>{~ zk>J;JBA^z*4`t`x^!(c<#;BYEr?kY$MLu9Vz+5B6NbYmSEfg3*RtOmJ|EWE#Ry#W} zTUVM@XA{AC6spQlLB8xNzalsh1xAo3#2D?69?IVI{daS`ltmfMafb((Fq)}Q&9vmZ zQ*o4Cmtx&x6?7*$D8)H<4yN^P6+0C^bKsNNcN{)uF!pS#o>!Q25 zenx>2)O$o2b-TM*B*H6%lfVH_o|*Bm?_%7 zP6$O&hK1fErU)o^CcG#`@Z&NOQ9a;CDn+x!6+@sDaZpfD80kCca~m0(=sQ|Da9cYN zuI{wew{x)P*0-{nGeEFGASk?MvOp$8u48#u+?st`oPS;X0H#H^$b~4nQMCgUB6vjx z{U8SD9rrGgdykby2d9K-U7Ia5F#)vwsLxhj?oGwz)LsM1wO3A}0!T=Vj#+X04(IRz zn&Nd0LO|KY6~a8YfE;&T0k(a?-ulARm;qX5{2N-fhtI6Rz9YiNA1DoTbcvZYFJO1B zkpjeb=;1>UAb1g4uriNOZjwS?Ptm@l%>}V$)qgg8g{}g2C%>5^@@y*)h!A{cz2FHk zL{1h?->-R3+K>5+bkLUN;`S@_g*bQ>8{jcP{k_)M%iY<-eH zrDPD;lb^gS{zTo$Wz=3hnl~J3%Ed54WNpy30(mT3=shBgnt=6I_eE8R6;?rbd4WPz zKqJLylSV@$@>sTDWj>buz9BR{c_AqHZQUx*1oql+u-;h|dErGGE}Bjh6hV)Rn4+ke zSAjGdhWk&m+5WEV@W7T>cU-@&*uTOFxQm5apLq?XCn_o8b$e;6d#OW@u4+dd^Yy=t zIDGG8wT-qq3>uAHRHCax_=7Q1RQaVGMPj}1nusXkF!Ay-SgIHa0Y?jgMoN+EVFMJ2 z^_+!yD+G${O|uG~omU7m99mlx`Q0w^6v`Hmb=a$y5t3c7(t#8V?=Fq=5Wo$ zD=&O&zH(j`QDnp7FY4j3N^mU%ZYHms zx}Di{S(Wocv>^Q(rybbGXZ+gbVJO51>OCTibb#g7iX{&7d4W66=Nc(S%Xt6#powXq z6#_;X9)HRvor^EY+OyllT+ddYfwskdB~Od)Z0U_ap+?a2BF1Rymhj~-K9nboTJoFy zEg1G=!idVuHF&0X7mF&6Do*^TdlxE7LbY1rE^OlkXv_WUrz^ff3{hF-AJLiO_#c`6<~m1gMg5GhWk{- zQ{nys4z2Y3xRLJZHgFpn9w<>lGRE(6j)wNcA{e0k^Me>51(*CC;ihWpyRQdx#Ppfn zz@$T(-=}-q^P^vBr;!dHd#m4>=R25E-VWohF zdy@WN9?SswJn@8zz#$^F3wB`rm@MS(+rVoO@#Tp-r zUOh@}jy<&wbFM{G0v75$B8-}V-5)wjR3TvL>p#}d!q?YFfDpl}5NLyd5N-3&*?}g8 zp}2=xj&_X`=5f&Zj(GCr`^ePy55PbMuR@?7!~m^{(Kn}-h<5*U1Rv_n|GXO$KuQzQ zR6k>+?!H~-pImPccmS0OIjtNRD}8(6WVj;b1sRjAkvKeK0K?T9SrHMo9k_#6JFF}* zs}SXeYtT_ecohQsjtC!B;FI3GR22fM@$mwURE0KDAG!nrgg}KLEAzRK2h))ceQ#EG z#c@qs%W5dl2R9J;29l@L$AkncL4**D%pgyQA)1&Kf1CEfwz-&xGx%!{KYk@2ao8d;L4JQqmW_Zc1s) z5cNb`jG!2qLA^(WQ4?@0Io(oIYGGh#4m47XR^UD9&|m~wAz(C=n0;;s-{8kqN}5;v z{loXpfpyS)*O6m%eK!M|W7E*{BE~4HA;0jKb74A`K}3p7&oVK{VD3PcA%5QD4`@Tq-vaptb%uw!5ghpOSA`_3My z4E{=TeA2&jGK2Ua2x5>Y#0d4Z22GFfT3o(m@hI6ZbRq^5LaDRgWr8BtGt!^49G&=~ z7mo^|A)3P5X`^cgd#T%RUrcW|#o@>D+aE9<));2wU7orO*R=~XLOauR(bWEcdXEUB zCSX7>zNq>jSnJ^72a1Z2Z)SXT^so{Nj36rnjA%EB%nm=RJ99^{a($$dR^#&&dB01S zE>Zf?eI#Epd)Dz0)ukNZ^CHIR+pplv`q+A=J32!b)6Ha`V8SSW%LbFIL#OE0%0)}A zH=IyJg^@caUA9&8@uHVQD-twBQr&TQflWN-59ujgjrU#>ru}kH95Y5M?sB3jqX)ew zB8(J(+8+*!suBFW5HIjn$RV;cHrkh3ATUWke z9eXk=ex(0WJegg=wl3tw2i;7WwQ)E+bM&OuNozkB^Mki`J1u`QfSI4K^urgxBnUnS z0Q-)Jo=#wzPIsyF%vGczi)1%ad{saRLO{tZyz*ELok>K1&%eGS#s|3O z?%bmIAc$eYz$_imNK>KhKW)oFfDoiYkQD+#ohO^}KlGYN_#2m13>6f__zV84<5|Q` zzS3HLWjhy$5Q0<)@`M}z3 zr9n}Cqc@wIM=xp|5@@_B;=U1wf22merKdjaXSmVTX6rk)xtJk({y@$Q)F1@<4fP%o zMmj(#*rcVVLO`8#UZ9a;WcWy&5(P$(6#_=$<#+h|s=gf3kNln+v$DZL!W_KP#fN-6 zpKEExjshdd6Jm^L=z7-mpRV3;m?=8HR5Cst6Gr*5v+0|z9?RQuvL?Y@q)rSKM#sKI zzh=Ig#K~S~d5c}RB^ZZS+8d&OWT!4mA=Sg1ryg_*W5!4?Y19SJ(KUs__Mf!+9# zwGIbxzkD_W=Q`52&kQW@PY}wJs)!%FLDP$wpSL;nbRa+Qu?*}xB6@~^Vg1fUV;KT^ z`1FLySLt`eAB1Y-!>2-k0fG$zJsdi0)>-4JDK9ULPBZ8yq>q;Y;L4zI%bdL>j6K~ z2-d|c-2+B2czi$`1bmX7=VbgUe){A%t+CW$C*B9In!yd!*U00SQR{;`az@ z&*1Rf*AkZbZ#QxAGe|iS%ir@7Gd?C`wWi4N0sD>!9|d3o_2DJr!wXzPgOQ$oiDQ|XY~~-kKbb?V;I3C3aD^jD?4s|sfU+NW@d4c+rf25C%9svm zSvFtZ-*)~DPflVIpEvSqvze0TH_PGqn1d>lAEKhiI$5rOkw)2XowpJ5aAfsiplTjn z(eDh}hjE5Xk?|r&pYUR)=da0Q6x;8?z9XWC*1(H`X^Hgk^XLITQt{cPaKs6@_<%MD z^bB!V>y17aaQZxyrCKbZ#Bc_>%+8Knd^CQ#^dT1?&<|pKybDVB+T{va4>dVQovnS_ zgb5$dho|IQuX}FhmT=2uG0$a3g^$~@tF;cM?v=|o-}ZQE-#>`Mf4DF9I^c}>?(MYc z$;^dQI+*cseZ-Aoa}d~fMEC>)1z<%N#fKGk{*Q0&v_8;C0rEaCfTGOVf|dF7r@nnE zXLfy2>SL!>BMAymrJ!YbadLzjGL2|Kgy4tq3!V@|^ojdr(}V79zzxcT2(R_d@RxVU!1qXq*=v#=~dOLOgoFj})S14OdnnS0T^_0ioJ)d!>=L zv8%#m8>$1eMvqLN0r!_Rl5Y-buvlFNA_U(FpdZ8l^=tRhY&|Aq5TuL4=XNWtS$pMhV1!_0qT5f? zSj+$&5ATHLf#3&VVBZnplK~9C=oVEWczpQz^nf2JJ`D}^50R%ppbY{(8Ezl{%1@j% zYwOA5O>`}f;09MR4qhTF*O^+!+Ju}Q&<$dGJa!Ii+DISCGoRs@`qtRnfr%c`xRxPZ zpR!^4`VA|*IluX%qNjYrsPad%onEi04rC{WcJTxEl6h#Ros!5Mi#bzYHm>RAftj8Q zC;u!%o2v@g+P$Kg-B`3d#!>2IxPPQIi{Eo_0N>@?&FF+}yJOq5>Rmt}u?Vc~PeNFo&zL`O}z%vYu9HK&K&=L@fS zqKpb6>hDZh_REGg^D8&A+|dqRfy0Zw-=}c*W6D*V2fM|r?uOmNEJ7^j*djrQ;6K2* zH$DHhi7;{pP99b-DnjtFj8_EM)B!Y7jCg(Mp&MP`W7&e0`E2OEa{8;>ERCQi#U3%^ zUBLt3gpj^CIYwu{G~ED|2tJlActVVk!|>OaQn8N*Pu`XLk+)WC0A>UtQD-hZskJ8ZkU9MmEX*r@@)0Kos2m7#!*t7_empNci7z1wAMzs$?du`i=)` z4_2X4ArDJMc9j)(3tL%ChD~$teZ}Eh{SLGG-8=sK&Yzfmo^VI(cT1Hh{e|ki!he7Z zZ<`1s9iRwi@1iP%Aeiw3n>&C;iqW5SatFbj8UaR-6#_< zA(&O00lS(=vTDZ%%e_%h1UW)XQHlJD@YlBsy4UM#NnObQCXR`s5XBAaKmHNh_%oIA zPuw?FZB!J|8|B#eCHCwOS@xXuN6uRp9NtE#-X?kd<-TO9HpV0L@l2Sd$l|8hVHB|p z>OCTgEPRV%V)>ZtlMN>{9_v{W{en%_kBWv5!8D`7>NTl z7v+{JMxuZi0gV)+b4Jv$XfOh;5HPZLy8r0Av+;eJCU40}U0vZJaDb*SO^%WJc$5PQ zHG-ZOF-DEC`?y@h@dY~hqgvb6f5Ud6%SK%2YN<&_7Im5$zB9cmyisAqdb=*0-|_O` z>+0clpNhN`;N)0Q^E>4dKGpYOn!73b^$uai$o_xx*h-p(x#fTqJ zBcPFDEAy$5hxYgM;^!P+Cc}3#XNmi71NRHb{Kx(EcTf08puh<7gczfe z(5SMjCuu_Y?6*D=6}^&)DX6V|X?L70UGLAQYw*gcu`5*OlMv z4BhCjUNrB~%1#Nwgi(Q=qK`LkOBP3tN^_X!g$Ps_6+H0%b8~7;y)H?!cl~G;P)Imb zt%j|@@H2nLbikUH0lV{iA=ULglSStRxqLL=xfp2)DR~SSH{#yc{6)`}Ul0#NZ z^$KqZ3}n3DZ4nAmIgn$AeuL7;V%jv;*j}|NoxaJmP`;K1gSOT2{A%uztJuIqqS{$?$b|Pu#=d zGlEKJlva7TTBMI(uRs398#6=&tL#^U5W#5&Jrmn$7^ zfn$95)AaKe2>6Kac<1OXIuc!dN+a%1%%OZvaOeGNKrczg_xlwnp{g+OAJM!M#N?du z2oABZ;O0F0Znh#Esce1sOIh7~sM@CpLO9e_qE zKHjpFPf&;tWQ9Oa?`-Dpp!7nEOHU2NvqX~>?n2Ac(kFe4=2V|5Kg2!Z%Oy+?#mII#81e$f;N9wR;;;BHKykz!Qc zNmUC95PS?kFN5*+^2gGv!0(~`Hi zGu#}9&i%!HMOmZ|wJ&mv7J82eqh13qFRrD=G=5;=3}~bnt*U2XM}rY)g@930b9%FZ zckk`nK2MW2?Ng8Jfnu5yIYv}tXQ3VW2oh@Oc@bkoIXND(Ta3-(0CON!M#mFdOfl^W zQ%IyxP}cDPzJ!D=RxeJW!pNJxkAG@h#_q(|9xI-wn@wen)k zm<%>~i?P}`W=SxneIP&5dWX*w+EI;wBFGV9ikLV3X|7RYX3Dag_PB6JuMHDL^i%7U zc1&4&+a4dLtFyl(kBXx01$1$joX=|JHI7DUns|=k@IBV%sTW3yX^hps8NUc@YQs#? z^!MMiD8vZrJtB&Ffop#mmr4;YkYEFilp?AExds$s1X&?aR1uc$F&lYr;Ax+GMvunZ zTTQn?jFic5j#idvL=)2>Plz$<-n>icOYD{GZ#$|>pG7fX+e_4-`2DKsNJ;souUpu{ z)j0Z4VWb^>Woz}aCuX^Mt2Yds`l64+JH9C$o$4Nv_T04JJ=pDvI%bTbI7Xq16A>iX zQ1208!~tMLvqX&efJv9RMv76*OD7cTuM1WP81eUrMmbDO^tqnQi2JN3nb=(e9&ybg zzdF(39W$zHroiV#jL{Lt!}^Y|qIPV^VsUVeWjKgQjq0*GU8y3o)t~=4tg-d*!*i%G zx>;UxF=FM7zTL-MXfDrW{>9;cPfnZ5p1#7--t+Zyhb2olW{f0XorjLt!(+7Yo`^8Y z1GZOZEj7ay02a=GMv75*Vq7%}HG-@VFw*fTjj=A+eD&M=zEX$G>>bac^;bUf_194e z3L_L5HslF0Miu&d)(`nLH(C^O6uh~5xfc^gRlASPB*~b{naYa!q?*eAL4{Gyg}-|a z-?lIO%6R47#cvs#aQJqsJR9Nq_$SQM)w6r=pP0prQEOx@G+_;o(cGJ!f7?VDg#&lq z1uwe(3cun3l(+tmTIOM1ZBZaZ9I!J_pbY{-3fesergmMgtn3}9oQ7J()uFXl3UY+< zFGQo+rvL9*5d##}&wA$LiFbFTZ1sXYdk*?y0!Xwa$hLFT(RWLO1UJn^2|Ox*{C^c4 z>fRSJb0K*{8qHMF0USOqp)@gk{ql34lVk_3xIW*H8K5!S;`3lu1V32|z9S-h)|~b7 zqF8b;12%5+l4sU_VfVy9fDp_U0T$-15D>~^cyCa#CED6`vf`e>pr!XyP=(rQ$Pr>b zUHT2V3e9<$_kHk@t0II+{`z@|e_*w*@_qmrWiVYKXu z;f*{cDNn_-2G2&}@Hs2G8qTlm%h8wUJN;?!#&gUR^*?%mrkFX@dqfmf0GT!Y62*v@ z4|pv=Bc>kA6#cUDBfT#wii~?B8-iHcQfkl$jD;3(K-_BrR-jrG6< zWgU6i`=yjSingf*@;x&6idlgUk55D8&)`?_=X;I_8$;m4#FBUC^8wfY0*y3=(dOSs zLleV5D+GLWY~6Hcd?;mkm1yefJRD?Yz)UFW7Fj05v*$=VNDqRkYUpth6BHE9_gJWn z*YD2|Wz>pY%VjVT#Br>IK|@s1qD#%QI!LQzFDin%r(XQ5qc&~9X$EooJ8;Y3@cDx` zGt+E-9I0sZKa%q{2m7&Chx7szXYc1;^!&>vqR0~14tr!#@nMA>k@E@wcjp0(l%hxN zEe}8ef*+PGSRqgp?zA`m>?fbAym$8`o#=iqr2)+`dXkIKL5~}@C@_Ma7coZZQd{lL z^L)|iw^dp%p3Z$A6GpschbrSKj+C-Dncps;eprPHqvDNK^d|fEZM%}stu8JQWrM@N z%rfZ^Ih~_-?><#IeO=dD%tF*r>WJoy2h@8+81(`h(WI7&5ic<71T<2N=q)2(pb#U- z3IQWSmET)8-K}!EPkYXx!lYuj6ild7&Cn)NhV&k{Epu4H2KNsqsYj-s%ZYjl5$64LoDh87wgJjo2yBHARe^Y*v( znxVXhR3xPC`eCLhHCG@7MJ5FG9uY+xKqho@sT2tTXYheWO3^85hj2!t!X!rP zbstVp4E##9J9l${dL!p0RAQ9#sa_~$t5$%>CesRPmY2YNcdmAmPZAy$O8(6$chh5t zOvH@Q!E5%3C@_M0j|d|jV5ZuC(V1%an8wE|3QXORXGY;4MW&Hsv|xpRQ67)1vG?1C zCTfqEvRUcrV9zEnvCbkNmsv>+`y$6^!4qPPwyaIi3A@7ZYl5$Z`jh;dhnO(ROMNg} zNa1^phRHZSA$tqDvP9=`7a67tuHI>u$*p_R@DAADDEn8ytwG;`J!-Exev++H7BfbC z+!@doStL`9;M*p`C=a+qI%`pk;KyYmdgK#CHs&RdL4@GPW%CvY_%u21;cV!8TjS4v z?yXeP=gesru&kaJego<0XPMwO%<4DfZ|>GS=2+yVGHz(h0#t@wyY}JXj25}qQ@d%lr?q3h zfBb2+;&D)c;Kyb2Jx7F1=UFcb>P3SX{5lyho%0`aLOJTUL%>9c12!uMZ4l7ueMO&F z5&fJyUTQR_-uQ;-B$x*6DkDpS_J3o31d@Z`)A{$Ph}qc_eXeNyCu?Kcj-4Dn`%A4c zu~W(+-aIvR{+ngcadWkS-RPFeaY$*ydo@sY;OPNv5a_&d*txw?aTj=izSJgwrh^ zGtzVy0nYh{XbtsY;K2r0t^QOjU_ z;0|_{>isQMPl9~ujZs0={HSXmUfY`$PdgPOTdUZE!|T2FIJv#CmcOZKI~T{7Br41r zlvI%i#V-W<4ZSBKjPiiNvBsjQ4-$-k$iPd!vhgKbv=fYCNH7Ae5HK=y%lT3#pr9gh zQ!-jjGjc1oPkt1~4g4G>F2)+}52PHf!Vt}|gLQk>T4X@L-dni11Y0?1`KsN)fJcy-_ z<9o#4L3i~11}{_q#TCZ}^WPp?m(Epwiq1a?D5>XNm3f$vOV>HH>$?38++OV4@t1d~ zLEG^WWYquqjtHM%11}1;Me%`;WxT+}O>>P@g+>iFLOU1XW7&cg0z&Rns@~P(x+b4` zizehR1mc3ANyT;K2)Vf&5&;o{Pl*;hA%0#_%SAVy!K;vvpdRoeh3Lq0R_H_ryb6If2ndDG*gpAv{ZZ&us(=j5 zj1=8S@UBEEIr770x+8ZgL4@E{2=s#(prlxzWyQ}F6(y%n)!gtF!*=&LyUQfCw|}B| z{;QYPG4xwXPytl$%hh}L)8X*!w`M$R+7Dd9;itr{PpBlUwyBy`nz^aG8T$!E#s2$H zHUzIiVBZnpV-Jk$tfyd_f?^1D}S90!r=`*a_Aao%V%Xz9-h&03&4IB$ceWPDq96l z57>7^^aPuFQMfKj4}uuR2gER-k@6!caxDf-f)KSGZE7((3{cT<^1Mx#LA$E zMDW^Zna$)eY_!~XABYfw7zTMl4AGiehQU=j+#`(*0xh|XvP_sD8VxoqFU-5;6(^1&G#R57+Is;;*pCGXoG-|y~~K?GhyjnrorYaSB`aMae!0P z8-Vo}k|DiW*a%cT4?cu}ZV>a+$b85z?RMG=d!;`spP7CW!o&~L+nIg04y`Dyy;RRB zcjNvQR3a4EbD+Do;)2S{xaYk}&XRj@_~-J+znVufoA174UY*gnuNgBx^m2|U<`%)e zBcjJ2Cq@n8g@atjo76|0%G%Jhr&Du-XdhOdT-mgVp0UoWV z`j4aaR)z+Yp!mRR@Vpnq;IN!f;I&$QDsH(<)~WdDiF261p-jRXe(dTkcHjsp`rXJiUbj`+D#sa{jV;RTg9s2tFuJ1dycyhWiR_atU;M02k~M-buH!{&RA z2pdBnb}fFS9$s_=`SgGvY2-@njfWx^eC7k%AfS`Seep!@kITpKMM7!9Nds}L(2B+? z^0>8TCCvzk4!r7seh|a6SMkL~p&xsA^^<#NTv*1i)enw{@@zOehAVYCt?f;<<6|-^ zc>Le>hOk?0JpD${#JA|A2rUk8wK~=!JK5zFL#%pz!qtaknBjS`bMreeYQcYYVBZnp z!w=k0M!Q7S5!EB#$dMc!@e)jX;8kbd!aO>kb@sEXUaZl!?M;h!UvK#oM2GS}%DwJ8 z#{tbj!~gm7UJ!$G!cnCs|LsGmv+Ir@>}Pwt4U_7Ks=oWMyYc1m(@u;V!hC7bO&s=k zuJ#P8J7Wgtk!}zd2oAjJ%=a7-Hl4tD zYw}xs5I^^sgk$$y8F;)fu~X2f;>?4;bF0)}E!?_M6uMX@xB6?JTz4gY6 ziVpmEix03n*GT#4a1?@WriUMIEm)aPdBQCdKWVh`2u>d|Y`tF>R|_qp3X;dI_;lK8 z5Fz;S)`BO*5J{MH)>WTjNQ}+6sK|G22e!ki)%rbd`?7m9o0qK~&Qp!@L>0Msj&j*N zs#g4|Gr%8iecZJQhmXEtWI5s;hsSA0U?FYCADose`|NVWJo(PaOiJ}HI2d9vuzX|RQBX^ zhF1Fbq43N0A1b~a03JgUWFE{%*V0XhQ=JG-$yEwV|%xJy(-cIra(oT$PqI5 z%FcmYg%&&^hRE;rudNTJSlKjh(WiU08)6r+^5PwvPunzJx(g@DmiQ-8!WA6|0{^UR6T%``h(z%|q&N^*<_ z89MeL$7sP5VvMFj1DmEkoAs+sEpt!a$SH{lBa3XWnA0V50FLKi^)JwK{ebQX^h+jQsLf7b3@K!4qPP zxO85rIBnItRZ#ob{m@v`A50h>QkE^g@aL()E;`NaJ=Phes4!Zu^!4PB$+jzEx6<#n zyq+os>b)lZ+;VQM?wK6^^7le};w@RP+~oh#Zw0m;uAB#c zdHSqb-A4Dx>fQ$b_(W6yJqum?BfL9oYNGsTDg@94h&%M)) zsc*6q#Am;RG#=*|0%Ms=A=#Shefq>5$SGQIgqWi0Yw?$hZcxQP9#M-6iF>;S6GacM zO$`kQREK16YfPRf{$fD>H7bufA95C?JCF+2W%_@Z4d}j=-i+FLu*r{ z#L%byb49WZn$XsuK(ehtpPX3DL4x4FB+w0Fe$*V^h4ysv7Q1Y?t($(zo(B^@*A>)- zr{CTTF!dNIY0OB-E355HiiKCva=P-#RNhi}tcF|_=p&*_TPLgr`rA##}c zVc=VP7gQnmPzLrL5k0EFWiEn?hBA11fO4O}k2H{#{=9Mv3}o>1fHny9(B5mjv!}g& z)ZQt8v8&FxS_s@=e_4urn)-&DzW{Q4KtG7_sr|VuO4V`O#4;IrmXPN=OEBSMLLcDf zPg`OsE5y9_aKN1xsPGYcrX^eTH2b-MK}*F%-Az|RUPN0zj^quY54^Vy(WI~V?0z#@cT+WwY$@I7U zoBOW*Wx3==a9hx``(y$%MiV{?5`-WTf*c`c=s|hKBl+)r{gqaH0aW_aH!(4E>VSf@ z{>S{?^3zxUc#c`Np<<}sgzH?-XWeMbE!%B8qf?g^r{}7djA7#0%_C1R5ztZaS7iC@6xg5Gd-Mtlk*+yK7*i zZR_7#uCu-M;Cfk<7a2u;o;T%DPy{(bOp$U(8`tZcDSWob=@a`Obz{3zGg|Q7$<#)t z)@wFd6X)!b#!ykjeyhnxg~vI{r~a&t7VWYI9Da?VqqQn^9eYfkvfK;BC>6{~)NCFY z2`UkM5-|6o=U+AvMI69EpJj_`5qu~U0nX!-&o2hQ<8c5Pf)8b&4FW-yp(#dq(adv zMmY~S|N6X%$=B@4+A1?t{AgSpJkJteR@7Xv=a+BQbw(UMM(fMCcz?7r|MEZ&zk2oJH5aED zF*B4-*9mPlga7^(dXI=Aci^3HFB-|VYNuzX7`y@+`zEaVDHLPMK4f z0pk4}CW2gqz`i5GCj$ryYZetD1o(*P36h5}1Ciw4DDe4bgMd$ln5)O&RjLWflP9#C zBGrU_!FlP+|8Yp8vrTU`3Vio`Q2PV^5*`$5yeXpMElZe57uQ>M<19Z#~ zK867=ned*8@iFkC;8-F)z=c%6k5qjcE!hqr#|N}Qz=!`Y|KM1{bX>JtPo&t?)A--e zt>Wv**H;4y^A01&2lRs&AK%dAm!<-*_B+d46funWpT>kwiqfG0-1JRD%e?%QFQ!xR zsPM^95xBWm;x`=+W7n^$tefmOeBhe*UIK6K{}_Lo9$}I*gT3U!l~2o2ENg&$M}$u; zpg!vs#RonG;uQeuCjyNWAh|&tiuvdTD+Gi(*PKvw-~Bl4g-V2HfttI%1{lAF`N`r} z_$qcZgL&w25i`VR8u9q}HM2KAZsk4ItO>z(%EwYT$6M>Jqdfy-`YyrG%p9l~%1jxq zk$rep*VZq^LU#X-1fZ5{XmNd3dg5d2 zER`Z2U~vp+q!iIVKG}H`n2A0GG%=tpzPdVRVcv?pv7NcTmARX~gSpK)E(?2`b4lvQ zs;b#4D>>BDucxY)SG`wINk3^Puc)xUl1=5KO1Z*&#TBd7yuxB~2H7^OdSpQ9cR=NW z=uJaSx}+y+q1tK>?nSN)NnAlPf@5lJ8pZ+7HP<9iJfF3(gZ|v#qUU66XFvCd4lWKn z$_kXrH&HCZ<0-hsXEj-UT#G>kh95&L{ON=b$X@DOy2Ioa=RGOkI4&wBuqc`R+qCKX zA3N?cGm9Bk0}nsfWKsMP|5my63~j7!?Tqc`{tn>(M#hG7e~paNVObg0lLLjfK_a$DOcyw(}=WH&Iah!yD*6=pTaMDGHR{0~)F9y~*!aM)B=KRtVp|`+KHy zJEx~day;4OawV$8bZx+Ie>Zt0bj6*8?I=D+$P?nv@knH0W_(1CvrOSK7rOWOI!vD< zv(@HuvDNx%>TB<%GJe0X4%O#)roOhgyj%NQ_4Iog#^*`nIDGx;j=eo4cqzI` z_5Q{DIc_}l9ZmFvdXMOH6bE+O$}PH@3y%>Wa97V{!%cozQ9N<)V`+ppHG{4LYt;xZw`hq9K7`gZqX4wAidiSI7R%fs4p>|9d z*>9Ho9mT(mO5u%PhJ)CiZ>TWRxm@G^uvc66$nuGjkL!*AbD5h5e%o8L6jVR^fQ$H) zl2(HmBeCPj+JJsg07ZEz;3aFJ_lPiZ2QX4vB1XKTz$81+NHOZ7^Uy=1MxYe}Mj7k$ zEdLD4KJ1uf3*DZgC#V5VCKTbwM}uy;Eg-Jc#{23EQ zr{7atf5Az$CnBftfx>|w=TTv_$?S<~*nV%R?LI5Z0xDm$;_%WgTC<{BTv58u_@#aK zQDQ#=X|*-t8*+>m-Zl|N;=rcum5X8oKOZG3pa=X&A=-u$rvj(MIRMTV&*VWH1caJu z{(ji~PO&m!x}4RyCdP3VX&^!HpAzT>F+Y7=PExI%4qa4&hqKZJ zwXr3ap)Ch*xy4^Ur{DhYZknKBGb(6n-hW7tfcNP?padbH=bsG% zJrz>78RKHI!mXdY=zghi(Nq-NTOzZTd~eB+G~NpZKL7k6#wTl>S@zed`Uo}lJEMla z>#<#J7SSoxXS6l;vV7`sVCxfS369e`n*ZVi2?C}&!87PAx3DKpHpb8*JC5Y9<4xH zN0&R85IQOLB9pd7%TRqfOV{ky5W2hZ!tV;x|0+wrC4v|7R+W^$fWuRGANMtoj`FUj zV5DAIN0EdXp~9EDp_()BVSl0bh%gF&M}ebVbR`!7MnXVF3N%uT4)WVV3WNY7$O-`? z9Ze7S8Xr0O61n8nc2ieOFM{hSzOv-wc^lERwJ0!xJR!zN`~021=hbM9Y!3O*ww1U4 z#)OgNZ@+R%+ChgAcjg(<4G!p5R3_w;jvcJye46))jmF!!eC-!>6OI>3d~{!7fDf#TTzQ3&0DkT`Ge zvg#rJF3!8(qUC)1o_(zcFA|z2M<{yBjm;oJ2)?C%&x#nJpsS|#DG4GyM~-$K5+69D zfeE0@Jia@k$@WE`Lk%3vlUF`R70SxfIP;545?_z-RE*~)r|!n#m%mr@KQ+erbt3py zM|(}icFX`3Nu7soEQe15z;{H1PbaXycFBt=e87?%&`4FNJ2n+fFoUcR5DFJLe|=cW zsjE9E;$p(s11;-15TSZeBI7HSi>FXQ#QnV@M@Vv`2jlqfxAY>4+`yG)@xOVWeHk*Bw(4(wv}LRh zGep9=5@Fyt48aEo^&SyMNx*?Mi>2m7Jiz^mKqJLy#7iR*)FA}H46;JNXsBoM`Pmk2 zmG9qkY3}waJ>v&cY8fH&lzMN&F-{a1L7os}w2|Vps@W2{4_eawZ>@t*|hIE>(ob2lRs&pKQ7@&SKZwrN>`xt9n;l+l2|AgO7CM-(+sF zE*|E3e&A!ra#Z+iwm8#~T4ihClrHCYxh--v4lgPm+fo-(%XLV4=Bbi39rh~LTsMs9 zLH&Wxg}}Ze!p9!Sg`5|~2Yy7x#}AZ{0U9Yl^5#_gK>dLaVhdIX2sNE4a2&fu8TzXr zbkK5W=APpV@YJ;~?*Y;o=~I`Rj(`dTp9?K`LJX1RO_ANJayGt7{Db!oDk;HXg6P*S zFL`eH9p(jBsr(KPH;15tC{!T1q-WkeN;{6d ziuIHDXs9R_{0F$udqfzyzXhtQEjl7YfDsQ6#ehbNQEMEfB?^onEAtrTCJkk_)qmf= z{<4Tf!OIhF zBPua!QTvd4IxIVNr;6wgzJih;z=gpxr`w;4HH`?I(un`Era%)jMg~KBOp#->(0fD} zH37E^xi1>i;0IU*>i>WIh(xVjx0-Y=)W4ey&V*y z4FL2c>-2Z{+MGdx-~}0UgP5Op*QRZ@Mr4?K2$WGY2feDn#LpEk`uvYkO-wDTdqX2u z<(Q!2=U3^UXbZXMlyUI?e9Zo%QVFnF3uRyCx{cKtP1RJd~rGe19yE`I{~flsW# zz9XV%4KS=fyVS&*2MA?ABju-qbIW>=9|WNcvNE3$jop@1d8=@&Fu*cO>A>%jU*Jv; z5%Rk|POLs=3lfANuZA2UX6UDQki*oZ%=dj7UhAGHux`P`kkY$NF}k8=(p+>BkJ@@V zs8BIE^d1pK zmcX7Np`~gOQ0ImZXrx-i)8>5^IYo08=B*GYnqq#DZLF9t` zQkJUH?x{G&KITlSy^lq3c=MSo#aV+AmhIZLY2Be?t(Yll>1mZlp+(SZBBIE>lma)f z#C%9dR1f%(GW438%M?tiIbic4&<24Zchd`9*+Mc|JFosOm3c5=ss!CPq(MH@nB13% zVx|%FgBYOoAH>XmS}Tbvt?7Dc@g=+!6F}W^D+He9?%JvCEo^qb_W?Q?T9w=zdfi6K zsgPy)*bn_a;21Xjstvy-DO?$DPT0I1)Vqj%)s-i?{WAyD%i9DK7g;K+?@eya9A8MMVPi_j7G)LxJw1fdLi*+dkD1MkFs(S!(oP$nd# z2mDAGN?Coa3uFj^2>r7`Ac*5$mR!1ZuSUTM#Z!@vToYa3F^ilZP-3-F>Xc@d5oH z#^-X2_<7|MRC|Mam{m^P`#FUPADV9?g?D3Q)I%N9_B1y8*rLKm`EBj#cHp`Y;vj~1lV_$OQYdZ z(A6BtQ{I|)o_-KBK0995pqLT@`;G{oV4xJlszvc(g-w2Wg@Ju%KqHM_@*Pb{AVBa_ zLJL+12!->U(ly8mu3pE`zmtx!U&J0-YyHGWHkvoq8=e3Wf}ayw@Prtm09z+l%hLwC zR>ra&y<$+c91}#*{O&6wj9T8QbOf)xu}1)12=ic|62Psm?3Jkc1Kf^4|>}~80i2hkmaH&5IjbF{Gz}(oIH&2Eb~7CCPDBR zEm$F7L|f^ue7o@bwzRWZzA4jNWV^vc+NGK-k-q1Y(vO^?1xJV}I`(e-P$}!4&wbWX z@~S7-U|X?|DCQEqCnazH+`d$=+o~ZSl@#?$Ts)Gs;lh)UqmFvoMk3E}_)mA9Ppm#K z+sCEaFTf)E_zq@@YTOylAWx(hdXI=Aci^22ERiB!Az))N&`8B-El1KWH;*HyXu%O;iWpjk#ce!R@onY&>O|>F`x28D zi8;1BkY3e5RW>wni97CM2P%qY_eJVe>s{G-CpX`1P5e`F9A0evgw$ATQ}>QEilJO8 zqh**WV(tz^F(!r1iJ?xpdKN}AOc;6ZFOA$QFxAm) ztsJ$vbKPxJ81XW!JzvVH!M)KVfnBGdj}nLHZGHOdMJrz)bDf$jr^)77%ou6B>pqTL zjOO0-{M#nNC>)qt99nd05k92x@&bj4fJQ1tX1M0tC@_Mo5HRYjSe^MLyYojCzhkx3 zsiXVuLtCjC$+uDu6oo_A!y?FxAWw)fvNPl?J9@^6Q{GSU^X+d!6__v@S^k8DsxN6~ zt@=3g$RCf2y}y(ADVsV4krkS|Ma7S;eE*l8w-Xfm_a5i2+rScp!ymZk<6>WO)8$~9QH{XQCvPzG<9s;e zHOLSAC_gAqjkscutJ>W+wLHPQ#+sNqwZ4l^DHJfl~FQ~k5@mjQY{r$6{ ztH2eqHlP>@$$pRh7pl3C(*wFeOpjW?gNQ9=WnG_n#M8oQ@GCIUvqrXBBj>(UH>=W@ zao`-_Jyi4v(oPt;-Ixqzo)lr3ayx+p#HA&iet2&gecJ~eH7JS+uy2wSj1AV|T{q@|skpF4WCU;1vq!`WHr zD#qR9^F!5geg{B=;6vDgC&Uo3xZRmn?WWP%qR=EMB(fFTA$zW}8-HY{j~RXU6M54% za}?d)#Y@YS=@U$fq)xod5c=GEZ~%vI^shY-tsyOy$1jp0Bcwi#S%S3opR5KY2wuJy zdXESrci?Di&{8uY0br{J&`4!y^ktPDa*XCI%v&K~B(B4_KRZXwX0~cF@9Mpx8xr6w zwePC`m3G!)RV`f@2fMqwK|$dhNxM6++as7Lhzg2gfvqTZ2dLO$cVc&8fO%Eyz9uSH zzd3qmAJ^UM%&fgXpEJ+>{`h>?@^{{|X7=m}^?>^RjWQZ;Ik!Y2+o(hi7stGhiUbYY z;~G=C(u;|@YLQ!!6W)b#H}klVX6K6??Xs5AREv5^& z`67Olh^f`^#h^~UHd`Zg+vuX>gJEldfXK&f`E%h9Su`#DeE*6iQSCf-Jqf%5tlEjf7JHhkaVH_hhAootfM|E(R0({I>>e zZ#_4WZXd4k+dht$6RJFT{p4rio<9zy9b0sOVxKZ<`=qY9M>(-rZXX_pY@f7IiOVOq zC|R}XOXD;Dt8-rHvQNd&Pd>)%{MmBohAyo#E}QGG$vz)8&dgK$OS)AT%bp+hJnX_G z+jOJyoG8@UxZNk^=Teiq6<@4>BS(1slNv_SoZTPxx!NPtrm4_Ags~7iSJlYce+?A3 zD&jOZ5MhQ|QTb(|@9X#YoP6r*V>)oRar?9PGIUWov7G8oY{WI&QM>{X#zIOYvWfQl zY^>n8+a~Gp+g*>x4r;3J(Ce#b?>ySpFwIy0_s=rsE_l1MCKIi`H|KqH+6rwa1Scnq z9GURWHr)lcxRyK5#q4xhU&!gc`474ILPG}uV_p8l}WVzv~2 zyYN+qd`!d+Gh(0r)}k|G`nhVD9{N|~mxaO`B@ZpVaP)+Ujw^SrY=5-y75=hpi~88k z!(NrQaRGZpP8N2aN*3-K4ooQY#6@bK$p-gG|LY>QHrv43e_eF&dHx+5x-v!Omy0@8*_-^T)9kcwyC!!} zbllz46wgnta>!rozo#|Wc~u{z!A43XvW>O|xm@=4i=2OI*1cDIqdc4IvQgH~7aGNl zOXZa3mLc!TMXx_;vQa0eho&+ChtDm){bNJ;$KBEF>1^7T>_sBaU8}t%qV$#t^Xus@ zMXQ`_G)yv@zv&)oJ9VGuL@i1inb`T1+5x)! zb8q1#O<*@_S~R?sSeKIZqxLGveI4 z_t(PRhK|$ipURpzBvRts5<+f|F?fHy7~A2 z&aX@}*a_)PxY=+dB zeC^o8BD3nH%e14y43CM+@^$RkVO-f3QTEl}f9ldGBIl{{A-Q|^>fO8kUyKGpE{ys^&^|SeQi%~UNiC4X>axK~Z;#>~L zqaJH&7Ao!dqu_}1PcoLwl#u$ysgc#9c6yCDx8cj3UJV-z%eMSO){Smo>jk~Ia65B{ z0?T`@3}3mr?df)|nTjOdPM<&5!1$Vjy~;*ZUO8sK-0_=g?W^t+d)59?iyX~P=h@h5 z@3s2VTW-$0Y~T9v%`Y#y+bxfKi%!pzH{P#4=gzZ3{VTMe6XBk}?3!F5esz{Rc58Vy z)6MbAH|MLJRQI2Hi`peRHYu?zX~E(HyYnnE_lE~P*zNqwn)US`^6-vF+hdD5 z9x!FNyCme!(sJ|89XJ}Wx^C7iDZ3Ot8Cu<@!Kn_1vJSc0WmNx|qw~img#H}aXx;wh zdnV%V57M>pEqC&U2xTx1XPh-NpOwb-WPY=-H3&|5^LFY*bkJ-m<$0xaO~(;CHf? z>#Sa_$hYIqYHtyqLly?LpVQz~X5cYrWs<=>y61nQ!Y=orX8uQkMS;Wp) zCU0%CbskJcdWtiwi*=D5wHe*Pya%wK~l3VimNC}oq0d7RJxM1|eZAT2~*+4H*c zzTMn|^FL8xx%$zW5TToowEz9b{JZp*1~8g_Kcqw8iLqOzbKqsUAp37r*k+<43j4&z zYP-0*q4{$8$w>T8R0z7UZ?&?#yEh;F&i_P(HS?#*6CuMgvQjE;zHJ`Eod3x1101sh z&zAi+D(op)69rw(v1?`y8d|Pde;Em$tPpkFG&gVAo~(?p zrY<9?GUOld=VPBGX}bZ^q!m>Z_}n)+z_||=>6sd&g~+>4=sbbD8-Ayk2GX-9LWd5w zA0!Ll?qXseHS)o$ciQ6)_akefz;`$OIsEXF!+S~P zjci)W+|;;_Rca3A=q)zfs6iD49($+YJPsUolz9frcvun^BEK;?Rw*jb;jM?zLlB`u zhsQ_*m>1#xenktr%JY+oZsSD-*6_c{nkev;6C1DOz#)we!7>^=St0JNuTM|L9X@v` zHGnSIucY#B?k4t8``c742+X-y{OvqUmNM5wVXyACG&K%;K3P}@yVd!k%FY2;POgSf zGbh4^<)qwjnY_WsPMTm%jk{dMTJyQL;kbK%22~Vz|1&m@95^f|w=_r#kvA$-AKboS zF-yLYbSAo>50nRRVUv}tsiEhM?(WaKObC6q22~V#_sgQIGA@=;;S*i4T2QbdO07Y!n8SXMqs<&Eq?!rasjFA<(^EnZeCkC7#9QAL3t zxu3H&2M){1Dh<*?%Yj20&&a|;+@tNnm6HWw$60Cul?D+ubog+oywQa@oHez>Hz%i>%yGjSenf*R z3jB4$QAas&=x~RLG9H$Mg~(r(ya;ZEw-goMN%Ro9pdXb6FtF!Lb5lc4*Y0dq9D2IR zva~I#DD>QECv?Z557Hnlgns5;6_0d`bT-!D4{6I3diJ`Yrw^9}FtU}e=B9?;;F?1? z4!yYsRTO$!kHSZA=zBFt3!(orWAb|(`j@G6Cc2<|O_K*;2S~d7-eL!zIZK_bXT|CB zG^nD`?_6jJ*1cGM`!*5ugoV)4jXwx>zgj|6n@$fwgbq8<`O*MvC(ys!i6HU~W$c#V zL&V!;O%!;;12s}`m&4zEtr;>JJXs;`uX9E#9)}(1sF~CNx?o?I%6k}1?4vF!pBe-V z$6@E6B}TUtp(Hox?nGq z%6qz**hlT}O?9`v#>>fFvL*_0dG3%8VY!LS0CDhD`uwm`avs5Z? zV7uj6Q+vCT?e{j^+we)Vvj$ZZc*l{OLpg9*yH9A479uaXLo5cZsn)Br@-_WXDA>#_c&RNmlWV7K>SQ?r|AZ61u56GMzFX^Scf zeALdIc{y-c!xw3g79yYjB>O7vZ}7j(mBX#h4CZb0Y5TMv9P9)}*RK^295 z(?6ebVjhHkUxTy|`iCInKpc9Jt#l?t=&*(dO9QYaD(EVen8Ilr;$>w&SrY}m+~CrQ zyBu1uJFq7}*vyb5lc)>}mS~hhBS^ENzP_ z3cX)cs_8iN7!A@w=%I1%SK!cJ?xtr?gbvF}ojtMu>{?y+Q6sO09~+;NafkPWAyd$meGONaA&7JC`*}Z zqOe!b-gKOo5(wL$EG&e5u1kOA!5+e1cZixf5jN~FZ4S%i4W4X)IBV)c(&NEIp83?w&O@^!>Sif;-6gw;2^y6J5}I z$^*Evd2!a%(BIyP*@;8npg|Rdo)}-+ghS7LO-e*bSO|T^-Mh+Wa0tECb$SS0&^O8h zu=j-q^zCNUx3f+7h#c1R_Zn1D=r&HCOK|9&Zb*qJ2@9dm$aP+sS%J_Ovh(L9d)t%{ zx}d+82QcrRLpKw4%D(S8?sSg?Y1)b^3jJ>CcFM6HaHzOKgR~I((Z=aI;^pQ3O?vja zpnKku24Dk413Fj?A2@h14t=x+RTO&E?pzmfr$5&qErkAY!dvB$2s+*F4xNcE=%Mle zp3Ld2scU*{?`5^{@^VsxDhmB!`+UmP@zCi`ccny>goV%txfU#rJKg^tJp>Utd?-38 z4Zt>AqnpErmig3;``m$#7TJHJ!ZQ3NOPXt&4eZXA~EjWlQEW~~M{Kjt_H+<~h zbf20z5w^Vo#(V}=>IX7;16u=PZfb{5^?Naev?8b*>CEqhy(iT+|_}U5+3vl4j;bSyN z3z2u)Y!kqdLx;ycr)N)u4jrEVg)D%Jn~8nY$Ommt(E@jP7qTV_-0RYdbsRXPae^!? z#Qk*FH+PO3I^6aZm4+_Z?WOV_?k4t8!)`t;q6-dt30V_`9X+ve3=TWhYZ(iktPu9% zup}#cfNH#<1|Y(=2cwkvQhD>)&*rAay)Km@gyV+g;XFWo}Rrf=%qeL1F(Bx4gcN^3zzFp&D=2wcY0q9swngl>ryFig`m@~ zX^<8|uRCr0dS1z)({p^HGa*8U9cMpj0Jel2T}7VaN~kh%>!6e@^AqQ0vL*`rrYfSZRyrohcvun^BF|QAiItOs7JQ|L&;|W(X#k^fWeKeMPyCw^t78xt;3-o*Px0*FZE*C zM;v;EpE4qrgoV((D!F9l6&*tNv$2)E5mK1wf__38z*v!;k;R%Cy6vSh-Z=D(wz9M> zswnivr*8z}&SP1=l-tsNE&mr{D>FFVKK~IneFtT+ltf`%zsirZM z7Z&)~UnGMxZABG@p492_avb_Z4bnpBWk>8)7CJ-dCo#gOww(Gi9MOAwq{8XisSXwqpg|v=aU9stxzK z1L#24ku_1^89#X_%blSGvt^ah;K>Sc=e)7o%6ThIvQYyNVZ#n|rBvSF$zC+Grgr$l zVgone4u7se6$Rd|a5LrHEZBjz$}Zz!Nmz*7=E(|WrUibdGjq^G=z{)28obhhF-Dmz6~tq=nGa<{E9K15L<9&z=Y!mK8&8Spc@I5?u!R zXw9u2cv%@t);R<<5S7o>;K1^pj+0CsMc0X;iwRd(C1 zIP}_uq-iUvDD+o$Ne(#l7!A@w=t}}ipW?*@RDlXoUZ;pO09#ODKo?ca+!Z$x zhdxDvDhj>+n;Ok==pQvm3!z`m(nHy%1)&>@(U}mT!wxiD8i4&+(c+T7>cWy7xdRxN zUm|Owz>{Ctd2`^9hI?@t4W6tJw|D9+Ww^&-2O3y{8h{8JcA#ga@&-3H$HJPrmVf-R zej&#VANRABlqGFZMS<68ylpQB4#(vo8l;8D*B?4+(* z8u_tXw;poj(BW;!nkeuWT{rm7$n zcV$bNSyOv@;kK(8IcV5HHqxMq!hVx9O*wuKc91(XNDG0Nn=;GFjkE8a=uC7$Z|p1$ z;K9b-tf`@Af97Uo!RJg3swniQ!zYDthePP!HAoAg+s?kNTu1}E22UfM2@yIRmCTX` zaIa`$A2ssUe{Y}1eGW$@H^`bO@G`XyhI8PMMzxAE8a!Dc?%jR;lwJC;W`(#=1L%T% zMJoU6g%1LIaia#=aM-zBWhrw_6!zy5^Wt&XBgw)-*keoGv~pYFUN>syx?pE>m&&t4 zC@^QA`Q&&Chuxa2iNX%))@LgY`v6&32z%*G z>id!V+yRVbi&v8+%{5Wr0pWgDo-8Mmg@w4+R-K`&GK3xGiR#qMiLhZ=DO5uyZ)7+A znVZ_-twy&iz;VN};-f(o1-|0O4!uKN84*juLg-b_rBjYUfKH!Zj~+r7^moz#Mz%oB z+|~%qRYbXm~FtWw^ ztf`^f4sX^0ujwN+sG`uz%{`J0hyFx^v=Dlk*TaDKglO@eHQQ!gX3vT1UA&mgCun_kY<6-3tAvjdr(43k%5jGqurtp%< z8`$v~=BCD7J#U^bxWgN0P(^{?yT0AZcJXZ*q=m@K4{dH`3G=%abS6aT(BTbR$^y8$ zo7hK<{A&f1vVa*nd>UC31>P;Qo0Vm6pUA>O+|?R%RL+Bg4mY%>($EEal2o4cnP|5; zJLBE9+}{xPd9o%7yW_KY$MACE(niLDCo6=#G)-qKOC$!gr3TOi`;=7Pt)hv2)Xomu zFf|sh-5J}-Qs$Z{>}&^eEXQFFA`1&)cd?JY#Gg5#vp2P;W=@0+Yj^4nGI=Ar(!ku* zxM!VCH1#U zi9^4sK^2AGF!k*9IP}_Gr9_m3h0srLsCyHK9%iD4&;|V#Apo1uLT_2lcKY2QUd>^7 zDb`J%HdjTVkE%A;%F^Y@8l;8L9W%RGS+fk!8_ddOW4fA{Cf!b03BdcITU4`BzIt_PI{5jN~VO;ULSLv3zq+{Ri< ztz6`>N`op2JhIzg&Kx-GK(qFg@vtN;MDFkNRJqv=ey2@)(L?BhzFHo@z!vATriPxY zU3>ywR$ggPMWN@Yki*KQsBL;ni6{vRpRkA=cE;ht=`1 z(t#R%rD-dwDD-`q!<9o2pwpLVkQPF(*zv4#(;Ia9t-kc^bwRJ#Pa1%oC1^lTK1^Zw zF`id**ntkypo&7Do9E>+yu2i8kQPGEk|MvA4Ohke=}dG%50M9OWxL#1Q#<|1&@R_- z=&>49QRo-ST}*>RFV|m6L`hf(-DPcFD?`OT0rU_==y23>L>hpdQHCB}EkwQ`_^OrNZEphUOo-5-!|Mmh0?v&XkE6atT!<~juX%JyUhfk2o8`zE?b5lD! zdzM;Z95;0Mc@3&4@J4Q~7w{VH7A)goNmz*dMRZ>)XV?Y}rH9Z3{Q@BX8zQ=6L&VhS zMi#`O=Nu+ao2#PGr<5#gWm()v4bnpBk6qW!$LCn~htRVpLWkQ@at)USa4})8-hQku z+M5H1+fv#R6;ap^FP~J7#fHE514LONX#4cB%Df8PmhyQ7jRO%fEGLae%ES%q?XS71 z%Squ{4XSa_u$;`)po+rY_PBfryqtX3AT0!5bZb{Dt86_-)0yakK1&|JnH`DEni~3{ z26dX_&=WMMqR_WIjI*)_zgDP}h?1}ndg}S*t!zsP4Wox3LWiT2oALnWW5^NYfsJS7 z<1;I8lu~$%H0ggeQQ$`o#vbJ^hoh8nWMLuhyM;a~chbU|6+4!ixh~jw$4TYQ%g7Pf z5it|2+#%74tck*&w)o3;9QHA?un_j45gC+aK@j%O@l+Z_*szATogkHWXAeEBsmn?J zX9LdS1C(YWJz*j89ZTw4IhDE2Bzg#4(3i>suwB0fbdPfPsLCmE z=!qIsQRw^n^|5jkcGJmHB1*zS=%x*ytz1nLK7}5F2p!h&N74Xn2#$`z&&_P`5U=6( z;j*N;CJOxTA)~F-@abe>A?_1}{Q*lJOdTkt{5Pz32NT|S54D))&)*bQe=1L%SsC6za@vud!TY7FPPRL9H6BeEt6 zyU_E#O5KJ-!N#*>EO@d)*gtx$u<{~#;%sUFU9j&-*i*5*8va)xT^cM-IQ!)+^~DbU}}i1~7QAc~jQZ&<*1YDMuc_vhqQL zDhj>Zt=kdY<*=-DiIx$uBrJqJ`S_e}IP}G<=pl&EVOjYo4ZtP{(K*7Pd=pb~pF4nc zC?2b2NpnpU_%Uza8yq;aU=&$ch`V^v)mG-e@2sI_PJ|81O2xG@c>}vr$=uWq|M~J; zE{+>Ie3%AR6!;Yvw+0+IbajvgO`=w8|fi*L64OOV3R=xbn0s8;TorL zr>BXLrmd)=(2HzpsLZE8rw`B|Ered=xZMpL`le0v>~%p;yICH9Io%U`!PdV=men}) zrW#aH=xvjh2jNcNtwCA{J+Ow4l|yVkZJ{&K1-;o;X@H8%>8z=1`tT)_-{8>aXi!C= z&u=!q3J(2;25BMm2)BmH4t!YCt8Aw;(FJ|3G=Pzvns07u=Ut94g+C24HvPq1WXVyZ7n{NA3WIip6%xlIEHy z@a_>W-8pb*!AWFcA@19Yz3%a94kxsZ@1|x>gbjy^1^3A04dw}c^mW9@v#KGFbgCak0!OdB+l4-?_q z!f2u*3fr-9N@ck+G++AtG7`TN6@qqisH>cr57!pfK0uQvLWa&>AQd;bvh}U3scZL` zx+z!U&c3HX6@}fSc=rI@*$ocLXjl>!0&n=jZ!#|?(Ag6X(L?Bh{+Bd>(U~2~#F`rV zrNlH>xw~OGDScR$wnY_%K6cjXBRKR34bnpBFR$cO?j(fJ&&SfU*9E=w5m^93MOMRE zQ$t_ZziMsV>HRdQqR?k=oHiAQeqDpK5PIbJEE902=R8Jdq6@koApo0Wsff+7q`zLO z91eY(22~V#nlIgym&>r@EOcB>geNS7o;R+h6{mMRK@Xt|`gVB$c6qe{y}6p!xdV9I z&v8kXG}lCd9~c#F<%M$ySy+hs%Yfqp`M3o>aqYTH&724uJ``oXB9k{7*b8UY)DEAt zU|}VW8;;LgX;4Lhm*{-z5$^DV8l;8DPgR<@pCgA3{~kwYq6>QKtFizFXEr`(O$|MC z$$l$uh8Ag1MWOF2-+BQKJ>@le!b0e8qLx_MG+XmJJ%ldki{t@}3_5FS=ym#5a>a*= z_cf@Z&==Z7x#7?o-jEVe5*9)qUUH;zK@D{J#CUoLUC{3n0|E)ivgpz9%XK#l!S%QAD$X)Kg-(f~#SJ203vb$MAF=aGy(?g*Hd~3B6K)fIV=s}VZz4b0lp_!aNuyXV*5;%@^3{H zwjsFJbq*UoiT5VT3PEqa)U*kI>V%_}<@=7Lep2)^#$D3z6-*e9pJ#>8=#VJWjrhi3z2U)w_hnG@CW4mjvhi6^nHW?te&6;1K--?naB$Y zgl_Y->^SRxk_9lb^`2NQA0MA^l_Pfm z9cMUM69wM6ul;Zi9Mbqe78c@8^{}<_Y8iH%PG6`rh_GQTpCFYtume8KP3`c{KG~Ib zKd`Kv*Px054;fm^g9C@P+$~AQ!;-KNd3^HTe{hEfeWiyWLWd5&APwN=YGNNX@_IY$ zm3R5j;n}~*lIEHy@UEZcDyMfshYuqQ3voAnvq-7p(BV73Q#02EJ5#bup6#$fcLW`- zv~)G^HX&>;vL*`q%Gc9#xw|3k-DF`Q>>-BN%5r@O`@=s}8bsKzoHY6&lQ$SyIbluh z?W!J=SMll%%gGE4swnW!XM5Ul;IN#0(;zKG?s4gdl}ndAY*N_t@DB2J>d~{8_h7#u zC;uS!+~;f>;5{fP(8=4+?-!5XOo-57IhiR9z_tOQJAv$@L)PLBzfRUff%jSPN!gMD z9bUy&MuR6S#9gTVU@H$lLsL)#5Mjf)lS@*0cN4ZYWu0GjW!EyCJIS6>mhf*y6trJ= zpOoC$@RuG;loew3+&58K2n6R&wx*(GO@s^`ohG$RoE`d)UQ)2>QB2f_q zZP(vHd7^|IUK3@7n4`K*ufQ=wN86>RaUep5jvgTuXRDOah02YyUY>^6=vcBQ3fkj- zz+2qWb{S+Oc(Ov&0S|U6&yet&?4FStfCw9oLiS1JjqD#_Zt79U!@+LK$%SwfVv|Xh zv_%yKzII#AOB^^Hh4j@REkwT2vzL`65Nk8jvnN7_4!6xB3*hQ%Vjnf~*WX%2^Fjh^ zc3rY23f#|oky1)v&5j`p3vsX6*itzN6*~NRRw@l5Z0PVB*<|tt7q+sBH8t+B*{4Qu z+|c0@G^nD$eeRaCa(?(b4bnp7+xssn%j-A%PAlY~GtmWoqBMZf$Tn87riLD~=EiUw z`UMTDDD=MzKP-)x75AJnB9?@O(CcT3x3W)ja4vcXUC=K|0~kD+(^*qPUq5}=0vvj- z+_JPSswnhH2QnPNp^wraErgymxSO&-89Mzy9(wk=py$pj4ZzOBGN5N-#TBoRnU7as zdFiA<6@@2RGL&QdfWIQYh3z5e(Df)^dhu`U>!t@Zjpg)iXFtXDy%}ot` z(#K~j@v>r9M3%Ni6@|X2(E0*6^cfnYh0qU9DQ;zj)y1Op?1|7}S+Of73&1udqx+IW z<62MT$YELON7h7v-xy-A+*$=K7)KTs;x25=q}*8r%Sx6KR2sTq_ms-Bjce%MwNuZ= zDfh@i*z3reDC~%ka2M`w2s?X884I4Q5cV@Wk2buZK-f)7Q3DWR!x|nfl{dJvGq6}w zmy^e3w>;tAhNF}h8dOo>Uvez|%z?ui-m0{Whb3Vl^8J~|TN#hfEJF{W3;GLr00Vo& z!?yB~H`;x?q2h z%X^qyv2x<$vGE-a+ud23^1qrW?8N(~3^?pavak@g)Az#4avuo$wtI{8{!@08_>T(-@t(WUX5*2VBhj`LqF z90DY1h=kf`=g#AWGqB{%tuFXS_;}F&5&tejGIbB|A26tAp!wgy{&$a_-7Wtb3%8SB z6FTxz0Y@E|G=vm7H&>~?%JU<1Zstn#e-+|f{*BZNHg+|C6Xw_b@Rffr4FQH$=2SIS_zm{&ANXegn}2^8 zf7r+d*3wP#0G@194QuM3_0a=wyzn2vM-8gzkDzJMBM)##cC8{Mq9iPIWcD_Xt<=&b zRp}uRN6Mjpk_Irc6DZ704LxlOdu2fmtUQ&g$R7YL0SmCQj?8V zt_?`6PS0K!^eQ!E0SumOPJ=Zy^rr`3tsFR{QK7bs22WOq zyZ@Z?$}J7Bv*=rg8bBB9BXW783H7(jn9lFHzai|@b)_l)tBJz?P(58J4%?qBEQCE^ zS3~8{3TL^{tx?<;ot{gR7P+5zph=MlMjw;SULk??+vO>&u z3sNY@*1^%$!v-`CM96S-;@(guZg61-60xQ(AUTugE#jbIM>$f1Dhm7Py1lD7Y}ipg z)gUbde&k?DWlc7$Ql%TyndpK(N*=(Cb(E~Bq1UW9Kyf#Oeq4hp3O&GcYbG3eg(gxW zO2R_u&GK!vGBxVglpcZz9d?u_VJ;=+yQiy>6=ND{#O$P9-Sv5HwO+a zIFKwX#C@#VK;@)BShHf9Q!^*RhGiv{msH-!CXiTDJN)_TO;*ZEBMqu3@UvexSUK}` zhX!dOa`&!P7O-bRn{2GX@AQ2OIul*c8@H4OaAwCHv8IOpDYRWA_c$yoGc~B9(96U( zD$iXGYx#E#(n9ECT0|-LxB9&*W6wrkV`E9!>`^F&b-0fs3b4?U> z>b`xHBaR^Kkz`>Z>vv=I5qwb9Cc1o)l)=tyUx3wpaw(g4n^p0K8d{@0q6{cz|@HK?M{ zJ4CMAheJ=>nVzr^`oa|Jtt=U?+l3xN7xZQF0PIzl5q;gYyZfgoK8pf7&PN(lQRwrW zj!nX$H|;7Vq9iPYp13WOJ1;KK>ER}N2wl)0Ndp+zQXF$rJH1p=t?9Va%XO2bZBa#` z=efRSJq~@225BL5pAV&bDd#Z!_kU^k1PPYD;m8n`g?p(9`17o@DLP0)Az@Y_ikcEY~pBx*j96$;?&>TIeG>EWa2kI-8H#)Pj!kXIQ56kDt%5lT8vRQ*F z3cOi{Mpn*z&DT@L!;-KN`NU#QR?h8i+lwAT7xc~200uTswnhn zIXrH2m&1pkuDxYMEC~yt=c_$dIm-e5ke2kJhtLK6lQe*lEsZxfHFW3uW4htcEBngQ zwy2`eJ9t&>fJ2YgAT5M`@5Nu$@ml_{FFkuAbXdzP_mc&1^E9!K8hNZ!h2FgTgtdG$ zSrY}m@%2tC4?|DM!b0568z0={xM3|X=})CWgbizXh*aL-%H~#BQ{#TQ@5L0{;jtQ2 zQQ)PD^lFYfylj6N4@<&AQtni7lLn$uRjTM-CmH zYJee{vs2)vm*kNQ% z6n5lzyTiD%pOb}!unWXqu`&)XJ%mbw2piV!5mI@BJ9`VLT zz0}Z5P5?Ca4UOl`VOUs&;|XNG=Q-pJCU9>HFW0`;q`d+hS1Xw zlcjA@MWHVo67vR!K1hSK5c-wjg}dM#=9Un8_PU^_A1(`EV8>UQn;Lp1hYiJW=*=~# zqR`jvonhs|i#-~oh0seppJrto{&@tQi7x2PM@j>*O&|tzBgnBR&y#q0nX5q+gBs-x*lbU~jd55R8zGoZKs^|8G(3x|G3gDMLB?XktmVPLSP z*9(;rQ4$tHe>Wz-GhSZCh0#M0p~La`U1;g)RjEl^{umT*e0?j3fpni zr3yIglVo8b?CMoZT3N4>Vgi*05jGqmc96;&&8MZK$E6QwvLFU8Cy^RdQQ(*APpZLz z!*Y^-qKt(BBszo5TU~w{!|)(O^u_o<0F<2%g&L*8txD-OPXt2*0l}y68h&XiHFI6C%S@BWyBJLDqxSdq)^iha*nP>GDD2cV_b7WP zA?!G^un=~5{gYNY%q-KXG>EWa4eu$HH?kdw=BCCSw<^NQJ@XqhsG`6}T(cd;-3@Da z?in&3mV|}K=gsb;tjB`iX{(v^5W1jmlm;+3v)yg1si9wd{@@AsIDE{1uR#@so)ns- zloSZP^DG$=OTt3v(_($P@uC8uFPcpcp$qy4c>s1otN}eGHYDl(N*uc99BJB$Dhj=S za@TUW(^qPc7D6vw&}$S9{lQ#%_C)BggBy07qsW>l@WWn} zw{hUmf{)3kWjrhi3z0uCbXCq=gWqY-Mf4E5pdXS4V6T%6=qu%e&-3rV%Sx)n z(zF#-6uO;P^+7oF{u-o(&{sLxSa}HAxP+cP5jrd@sh7$Eu)SC4_A9r@2439fu&gvB zYofqM*O}3c1BVveLKYU{{+#;Y3|_@yS$VaLN`nYHy)xO9&E{Fn+L0-2_~8>y{KN@t zH+gpO*ZThX*ZB1_{IB!l5sJ*u^pKq(W#gF2#-_OP*KKT~!WLqG{m + +HELPERS_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) + +src_backup_filename="$1" +src_backup_path="$HELPERS_DIR/../backups/${src_backup_filename}" + +backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") + +if [ -z "${backups_disk_root}" ]; then + echo "Disk 'backups' not found" + exit 1 +fi + +dest_relative_path=${CLICKHOUSE_DATABASE}/${src_backup_filename} +dest_path=${backups_disk_root}/${dest_relative_path} + +mkdir -p "$(dirname "${dest_path}")" +ln -s "${src_backup_path}" "${dest_path}" + +echo "${dest_relative_path}" From d421636a5fdc3b73fa5cb05e83529483d69e75e2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 09:26:09 +0200 Subject: [PATCH 254/496] Protect temporary part directories from removing during RESTORE. --- src/Storages/MergeTree/MergeTreeData.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e10f5a0227..ce27ad24e10 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5557,12 +5557,16 @@ public: auto it = temp_part_dirs.find(part_name); if (it == temp_part_dirs.end()) { - auto temp_part_dir = std::make_shared(disk, fs::path{storage->getRelativeDataPath()} / ("tmp_restore_" + part_name + "-")); + auto temp_dir_deleter = std::make_unique(disk, fs::path{storage->getRelativeDataPath()} / ("tmp_restore_" + part_name + "-")); + auto temp_part_dir = fs::path{temp_dir_deleter->getRelativePath()}.filename(); /// Attaching parts will rename them so it's expected for a temporary part directory not to exist anymore in the end. - temp_part_dir->setShowWarningIfRemoved(false); - it = temp_part_dirs.emplace(part_name, temp_part_dir).first; + temp_dir_deleter->setShowWarningIfRemoved(false); + /// The following holder is needed to prevent clearOldTemporaryDirectories() from clearing `temp_part_dir` before we attach the part. + auto temp_dir_holder = storage->getTemporaryPartDirectoryHolder(temp_part_dir); + it = temp_part_dirs.emplace(part_name, + std::make_pair(std::move(temp_dir_deleter), std::move(temp_dir_holder))).first; } - return it->second->getRelativePath(); + return it->second.first->getRelativePath(); } private: @@ -5588,7 +5592,7 @@ private: size_t num_parts = 0; size_t num_broken_parts = 0; MutableDataPartsVector parts; - std::map> temp_part_dirs; + std::map, scope_guard>> temp_part_dirs; mutable std::mutex mutex; }; From 4e2f8576e5a6e8e39a16334d0c697d5cb09e0469 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 11:50:58 +0200 Subject: [PATCH 255/496] Revert "Add settings to replace external engines to Null during create" --- docs/en/operations/settings/settings.md | 12 - src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 35 --- .../test_restore_external_engines/__init__.py | 0 .../configs/backups_disk.xml | 14 -- .../configs/remote_servers.xml | 21 -- .../test_restore_external_engines/test.py | 218 ------------------ 8 files changed, 1 insertion(+), 305 deletions(-) delete mode 100644 tests/integration/test_restore_external_engines/__init__.py delete mode 100644 tests/integration/test_restore_external_engines/configs/backups_disk.xml delete mode 100644 tests/integration/test_restore_external_engines/configs/remote_servers.xml delete mode 100644 tests/integration/test_restore_external_engines/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8739414464e..c3f697c3bdc 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,15 +5608,3 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. - -## restore_replace_external_engines_to_null - -For testing purposes. Replaces all external engines to Null to not initiate external connections. - -Default value: `False` - -## restore_replace_external_table_functions_to_null - -For testing purposes. Replaces all external table functions to Null to not initiate external connections. - -Default value: `False` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27b71558bd3..4fc2034b855 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -893,8 +893,6 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ - M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ - M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8bea0b1eed3..9faf77e9087 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,9 +80,7 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } - - void setNullTableEngine(ASTStorage & storage) - { - auto engine_ast = std::make_shared(); - engine_ast->name = "Null"; - engine_ast->no_empty_args = true; - storage.set(storage.engine, engine_ast); - } - } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.as_table_function) - { - if (getContext()->getSettingsRef().restore_replace_external_table_functions_to_null) - { - const auto & factory = TableFunctionFactory::instance(); - - auto properties = factory.tryGetProperties(create.as_table_function->as()->name); - if (properties && properties->allow_readonly) - return; - if (!create.storage) - { - auto storage_ast = std::make_shared(); - create.set(create.storage, storage_ast); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage should not be created yet, it's a bug."); - create.as_table_function = nullptr; - setNullTableEngine(*create.storage); - } return; - } if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) return; @@ -1043,13 +1015,6 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); } - /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to - /// Null table engine. - else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) - { - if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) - setNullTableEngine(*create.storage); - } return; } diff --git a/tests/integration/test_restore_external_engines/__init__.py b/tests/integration/test_restore_external_engines/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_restore_external_engines/configs/backups_disk.xml b/tests/integration/test_restore_external_engines/configs/backups_disk.xml deleted file mode 100644 index f7d666c6542..00000000000 --- a/tests/integration/test_restore_external_engines/configs/backups_disk.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - - local - /backups/ - - - - - backups - /backups/ - - diff --git a/tests/integration/test_restore_external_engines/configs/remote_servers.xml b/tests/integration/test_restore_external_engines/configs/remote_servers.xml deleted file mode 100644 index 76ad3618339..00000000000 --- a/tests/integration/test_restore_external_engines/configs/remote_servers.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - true - - replica1 - 9000 - - - replica2 - 9000 - - - replica3 - 9000 - - - - - diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py deleted file mode 100644 index cf189f2a6ed..00000000000 --- a/tests/integration/test_restore_external_engines/test.py +++ /dev/null @@ -1,218 +0,0 @@ -import pytest - -import pymysql.cursors -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] - -node1 = cluster.add_instance( - "replica1", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -node2 = cluster.add_instance( - "replica2", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -node3 = cluster.add_instance( - "replica3", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -nodes = [node1, node2, node3] - -backup_id_counter = 0 - - -def new_backup_name(): - global backup_id_counter - backup_id_counter += 1 - return f"Disk('backups', '{backup_id_counter}/')" - - -def cleanup_nodes(nodes, dbname): - for node in nodes: - node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") - - -def fill_nodes(nodes, dbname): - cleanup_nodes(nodes, dbname) - for node in nodes: - node.query( - f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')" - ) - - -def drop_mysql_table(conn, tableName): - with conn.cursor() as cursor: - cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") - - -def get_mysql_conn(cluster): - conn = pymysql.connect( - user="root", - password="clickhouse", - host=cluster.mysql8_ip, - port=cluster.mysql8_port, - ) - return conn - - -def fill_tables(cluster, dbname): - fill_nodes(nodes, dbname) - - conn = get_mysql_conn(cluster) - - with conn.cursor() as cursor: - cursor.execute("DROP DATABASE IF EXISTS clickhouse") - cursor.execute("CREATE DATABASE clickhouse") - cursor.execute("DROP TABLE IF EXISTS clickhouse.inference_table") - cursor.execute( - "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" - ) - cursor.execute( - "INSERT INTO clickhouse.inference_table VALUES (100, X'9fad5e9eefdfb449')" - ) - conn.commit() - - parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" - - node1.query( - f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" - ) - node1.query( - f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})" - ) - - node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") - node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") - - expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" - assert ( - node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") - == expected - ) - assert ( - node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") - == expected - ) - assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" - assert ( - node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" - ) - assert ( - node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") - == "100\n" - ) - assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - - except Exception as ex: - print(ex) - - finally: - cluster.shutdown() - - -def test_restore_table(start_cluster): - fill_tables(cluster, "replicated") - backup_name = new_backup_name() - node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - node2.query(f"BACKUP DATABASE replicated TO {backup_name}") - - node2.query("DROP TABLE replicated.mysql_schema_inference_engine") - node2.query("DROP TABLE replicated.mysql_schema_inference_function") - - node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" - assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" - - node3.query( - f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true" - ) - node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" - ) - == "1\t100\n" - ) - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function" - ) - == "1\t100\n" - ) - assert ( - node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" - ) - cleanup_nodes(nodes, "replicated") - - -def test_restore_table_null(start_cluster): - fill_tables(cluster, "replicated2") - - backup_name = new_backup_name() - node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") - - node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") - node2.query("DROP TABLE replicated2.mysql_schema_inference_function") - - node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" - assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - - node3.query( - f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1" - ) - node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" - ) - == "0\t0\n" - ) - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function" - ) - == "0\t0\n" - ) - assert ( - node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" - ) - assert ( - node1.query( - "SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'" - ) - == "Null\nNull\n" - ) - assert ( - node1.query( - "SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'" - ) - == "MergeTree\n" - ) - cleanup_nodes(nodes, "replicated2") From 06863cf4157765c04759109afa756022dc5e9c55 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 31 Jul 2024 12:12:30 +0200 Subject: [PATCH 256/496] fix for allow_experimental_analyzer --- tests/queries/0_stateless/00309_formats.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index 0366cdeea5c..b784907be08 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -12,5 +12,5 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SET enable_named_columns_in_function_tuple = 1; -SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes SETTINGS allow_experimental_analyzer=1; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes SETTINGS allow_experimental_analyzer=1; From debcc2e61053f763cb84e34e48275dbebd5bd544 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 10:46:19 +0200 Subject: [PATCH 257/496] Fix test test_mutation --- tests/integration/test_backup_restore_on_cluster/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 1b7f4aaa97d..d20e10e8a04 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1054,9 +1054,12 @@ def test_mutation(): backup_name = new_backup_name() node1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") - assert not has_mutation_in_backup("0000000000", backup_name, "default", "tbl") + # mutation #0000000000: "UPDATE x=x+1 WHERE 1" could already finish before starting the backup + # mutation #0000000001: "UPDATE x=x+1+sleep(3) WHERE 1" assert has_mutation_in_backup("0000000001", backup_name, "default", "tbl") + # mutation #0000000002: "UPDATE x=x+1+sleep(3) WHERE 1" assert has_mutation_in_backup("0000000002", backup_name, "default", "tbl") + # mutation #0000000003: not expected assert not has_mutation_in_backup("0000000003", backup_name, "default", "tbl") node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") From f9f17fb61e2ab27f90434b5e3fc9081c061eaae4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Jul 2024 12:32:17 +0200 Subject: [PATCH 258/496] Fix reference --- .../0_stateless/03215_parsing_archive_name_s3.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference index b4804c82dc2..b27524812c7 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -1,3 +1,3 @@ -::03215_archive.csv test/::03215_archive.csv -test::03215_archive.csv test/test::03215_archive.csv -test.zip::03215_archive.csv test/test.zip::03215_archive.csv +::03215_archive.csv test/::03215_archive.csv +test::03215_archive.csv test/test::03215_archive.csv +test.zip::03215_archive.csv test/test.zip::03215_archive.csv From c81d3322b18b0eb4b45b91ac019a8c4f42d7518d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 13:39:30 +0200 Subject: [PATCH 259/496] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index c29b604d23d..5c432350768 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-distributed-cache +# Tags: long, no-random-settings, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d5f8540fbbcc7e5f8014212d67804412c7466d7d Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 12:00:09 +0000 Subject: [PATCH 260/496] Fix --- tests/integration/helpers/cluster.py | 25 +++++++++++++------- tests/integration/helpers/retry_decorator.py | 7 ++++-- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7d80fbe90f8..7f0a9154be9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2692,11 +2692,13 @@ class ClickHouseCluster: images_pull_cmd = self.base_cmd + ["pull"] # sometimes dockerhub/proxy can be flaky - retry( - log_function=lambda exception: logging.info( - "Got exception pulling images: %s", exception - ), - )(run_and_check)(images_pull_cmd) + def logging_pulling_images(**kwargs): + if "exception" in kwargs: + logging.info( + "Got exception pulling images: %s", kwargs["exception"] + ) + + retry(log_function=logging_pulling_images)(run_and_check)(images_pull_cmd) if self.with_zookeeper_secure and self.base_zookeeper_cmd: logging.debug("Setup ZooKeeper Secure") @@ -2969,11 +2971,16 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - retry( - log_function=lambda exception: logging.info( + def logging_azurite_initialization(exception, retry_number, sleep_time): + logging.info( f"Azurite initialization failed with error: {exception}" - ), - )(run_and_check)(azurite_start_cmd) + ) + + retry( + log_function=logging_azurite_initialization, + )( + run_and_check + )(azurite_start_cmd) self.up_called = True logging.info("Trying to connect to Azurite") self.wait_azurite_to_start() diff --git a/tests/integration/helpers/retry_decorator.py b/tests/integration/helpers/retry_decorator.py index aaa040464c2..e7bafbe29c1 100644 --- a/tests/integration/helpers/retry_decorator.py +++ b/tests/integration/helpers/retry_decorator.py @@ -8,7 +8,7 @@ def retry( delay: float = 1, backoff: float = 1.5, jitter: float = 2, - log_function=lambda *args, **kwargs: None, + log_function=None, # should take **kwargs or arguments: `retry_number`, `exception` and `sleep_time` retriable_expections_list: List[Type[BaseException]] = [Exception], ): def inner(func): @@ -26,8 +26,11 @@ def retry( break if not should_retry or (retry == retries - 1): raise e - log_function(retry=retry, exception=e) sleep_time = current_delay + random.uniform(0, jitter) + if log_function is not None: + log_function( + retry_number=retry, exception=e, sleep_time=sleep_time + ) time.sleep(sleep_time) current_delay *= backoff From 15e0033016eb0e23a7e6f512d5096e50863e3187 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 31 Jul 2024 12:09:49 +0000 Subject: [PATCH 261/496] Bring back the strict check Also update the doc. --- tests/integration/README.md | 9 +++++---- tests/integration/test_storage_s3_queue/test.py | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index ab984b7bd04..a8deb97b526 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -124,10 +124,11 @@ You can just open shell inside a container by overwritting the command: ### Parallel test execution On the CI, we run a number of parallel runners (5 at the time of this writing), each on its own -Docker container. These runner containers spawn more containers for the services needed such as -ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized -using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` -to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the +Docker container. These runner containers spawn more containers for each test for the services +needed such as ZooKeeper, MySQL, PostgreSQL and minio, among others. This means that tests do not +share any services among them. Within each runner, tests are parallelized using +[pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` to +[distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the documentation words: this guarantees that all tests in a file run in the same worker. This means that any test within the same file will never execute their tests in parallel. They'll be executed on the same worker one after the other. diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index e3445d14cdb..9a97e8c23d1 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -907,7 +907,7 @@ def test_max_set_age(started_cluster): file_with_error = f"max_set_age_fail_{uuid4().hex[:8]}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) - wait_for_condition(lambda: failed_count + 1 <= get_object_storage_failures()) + wait_for_condition(lambda: failed_count + 1 == get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( @@ -920,7 +920,7 @@ def test_max_set_age(started_cluster): ) ) - wait_for_condition(lambda: failed_count + 2 <= get_object_storage_failures()) + wait_for_condition(lambda: failed_count + 2 == get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( From 9d14053cfe7867fd688c08b493c264ee679a4a61 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 Jul 2024 12:17:56 +0000 Subject: [PATCH 262/496] Proper fix for short circuit execution with nested dictGetOrDefaultt --- src/Columns/ColumnFunction.cpp | 26 ++++++++++++++----- src/Interpreters/ExpressionActions.cpp | 4 --- ...sted_short_circuit_functions_bug.reference | 2 ++ ...210_nested_short_circuit_functions_bug.sql | 3 +++ 4 files changed, 24 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference create mode 100644 tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index fc81efaac0c..18c343c6ca6 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -296,16 +296,28 @@ ColumnWithTypeAndName ColumnFunction::reduce() const function->getName(), toString(args), toString(captured)); ColumnsWithTypeAndName columns = captured_columns; - IFunction::ShortCircuitSettings settings; /// Arguments of lazy executed function can also be lazy executed. - /// But we shouldn't execute arguments if this function is short circuit, - /// because it will handle lazy executed arguments by itself. - if (is_short_circuit_argument && !function->isShortCircuit(settings, args)) + if (is_short_circuit_argument) { - for (auto & col : columns) + IFunction::ShortCircuitSettings settings; + /// We shouldn't execute all arguments if this function is short circuit, + /// because it will handle lazy executed arguments by itself. + /// Execute only arguments with disabled lazy execution. + if (function->isShortCircuit(settings, args)) { - if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(col.column)) - col = arg->reduce(); + for (size_t i : settings.arguments_with_disabled_lazy_execution) + { + if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(columns[i].column)) + columns[i] = arg->reduce(); + } + } + else + { + for (auto & col : columns) + { + if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(col.column)) + col = arg->reduce(); + } } } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index d832f568cb8..8993830af14 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -195,10 +195,6 @@ static void setLazyExecutionInfo( } lazy_execution_info.short_circuit_ancestors_info[parent].insert(indexes.begin(), indexes.end()); - /// After checking arguments_with_disabled_lazy_execution, if there is no relation with parent, - /// disable the current node. - if (indexes.empty()) - lazy_execution_info.can_be_lazy_executed = false; } else /// If lazy execution is disabled for one of parents, we should disable it for current node. diff --git a/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql new file mode 100644 index 00000000000..923f1e3be1f --- /dev/null +++ b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql @@ -0,0 +1,3 @@ +select if(equals(materialize('abc'), 'aws.lambda.duration'), if(toFloat64(materialize('x86_74')) < 50.0000, 0, 1), 0) settings short_circuit_function_evaluation='enable'; +select if(equals(materialize('abc'), 'aws.lambda.duration'), if(toFloat64(materialize('x86_74')) < 50.0000, 0, 1), 0) settings short_circuit_function_evaluation='force_enable'; + From 6b7c5eb5da1be1fc31d4ebfd4f0dfa0c6a6e728c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Jul 2024 14:09:07 +0200 Subject: [PATCH 263/496] Fix drop --- src/Storages/StorageKeeperMap.cpp | 34 +++++++++++++++-------- src/Storages/StorageKeeperMap.h | 21 ++++++++++---- tests/integration/test_keeper_map/test.py | 5 ++-- 3 files changed, 41 insertions(+), 19 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1559b442e43..0634c7be6ee 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -79,6 +79,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int LIMIT_EXCEEDED; extern const int CANNOT_RESTORE_TABLE; + extern const int INVALID_STATE; } namespace @@ -497,7 +498,7 @@ StorageKeeperMap::StorageKeeperMap( } - table_is_valid = true; + table_status = TableStatus::VALID; /// we are the first table created for the specified Keeper path, i.e. we are the first replica return; } @@ -656,7 +657,18 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E void StorageKeeperMap::drop() { - checkTable(); + auto current_table_status = getTableStatus(); + if (current_table_status == TableStatus::UNKNOWN) + { + static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " + "once a connection is established and metadata is verified"; + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + } + + /// if only column metadata is wrong we can still drop the table correctly + if (current_table_status == TableStatus::INVALID_KEEPER_STRUCTURE) + return; + auto client = getClient(); // we allow ZNONODE in case we got hardware error on previous drop @@ -1017,11 +1029,11 @@ UInt64 StorageKeeperMap::keysLimit() const return keys_limit; } -std::optional StorageKeeperMap::isTableValid() const +StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const { std::lock_guard lock{init_mutex}; - if (table_is_valid.has_value()) - return table_is_valid; + if (table_status != TableStatus::UNKNOWN) + return table_status; [&] { @@ -1034,7 +1046,7 @@ std::optional StorageKeeperMap::isTableValid() const if (metadata_stat.numChildren == 0) { - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; return; } @@ -1045,7 +1057,7 @@ std::optional StorageKeeperMap::isTableValid() const "Table definition does not match to the one stored in the path {}. Stored definition: {}", zk_root_path, stored_metadata_string); - table_is_valid = false; + table_status = TableStatus::INVALID_METADATA; return; } @@ -1058,7 +1070,7 @@ std::optional StorageKeeperMap::isTableValid() const Coordination::Responses responses; client->tryMulti(requests, responses); - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; if (responses[0]->error != Coordination::Error::ZOK) { LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); @@ -1077,18 +1089,18 @@ std::optional StorageKeeperMap::isTableValid() const return; } - table_is_valid = true; + table_status = TableStatus::VALID; } catch (const Coordination::Exception & e) { tryLogCurrentException(log); if (!Coordination::isHardwareError(e.code)) - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; } }(); - return table_is_valid; + return table_status; } Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index cfbb35ab2fe..8ed348a4f6f 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -80,8 +80,8 @@ public: template void checkTable() const { - auto is_table_valid = isTableValid(); - if (!is_table_valid.has_value()) + auto current_table_status = getTableStatus(); + if (table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " "once a connection is established and metadata is verified"; @@ -94,10 +94,10 @@ public: } } - if (!*is_table_valid) + if (current_table_status != TableStatus::VALID) { static constexpr auto error_msg - = "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table"; + = "Failed to activate table because of invalid metadata in ZooKeeper. Please DROP/DETACH table"; if constexpr (throw_on_error) throw Exception(ErrorCodes::INVALID_STATE, error_msg); else @@ -111,7 +111,15 @@ public: private: bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); - std::optional isTableValid() const; + enum class TableStatus : uint8_t + { + UNKNOWN, + INVALID_METADATA, + INVALID_KEEPER_STRUCTURE, + VALID + }; + + TableStatus getTableStatus() const; void restoreDataImpl( const BackupPtr & backup, @@ -143,7 +151,8 @@ private: mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; mutable std::mutex init_mutex; - mutable std::optional table_is_valid; + + mutable TableStatus table_status{TableStatus::UNKNOWN}; LoggerPtr log; }; diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 7aee5df5746..4b1bcd11cfe 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -67,6 +67,7 @@ def run_query(query): def test_keeper_map_without_zk(started_cluster): + run_query("DROP TABLE IF EXISTS test_keeper_map_without_zk SYNC") assert_keeper_exception_after_partition( "CREATE TABLE test_keeper_map_without_zk (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_without_zk') PRIMARY KEY(key);" ) @@ -107,12 +108,12 @@ def test_keeper_map_without_zk(started_cluster): ) assert "Failed to activate table because of invalid metadata in ZooKeeper" in error - node.query("DETACH TABLE test_keeper_map_without_zk") - client.stop() def test_keeper_map_with_failed_drop(started_cluster): + run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop SYNC") + run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop_another SYNC") run_query( "CREATE TABLE test_keeper_map_with_failed_drop (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_with_failed_drop') PRIMARY KEY(key);" ) From 406ac2279ecbfc24913548dfcf459c55dd450723 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 14:48:33 +0200 Subject: [PATCH 264/496] Analyzer: Do not traverse unresolved subtrees --- src/Planner/findParallelReplicasQuery.cpp | 12 +++++------- src/Planner/findQueryForParallelReplicas.h | 2 +- ...5_analyzer_replace_with_dummy_tables.reference | 0 .../03215_analyzer_replace_with_dummy_tables.sql | 15 +++++++++++++++ 4 files changed, 21 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index c89a70be541..1140f30ad9c 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -113,13 +113,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre return res; } -class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitor +class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitorWithContext { public: - using Base = InDepthQueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * table_node = node->as(); auto * table_function_node = node->as(); @@ -134,21 +134,19 @@ public: ColumnsDescription(storage_snapshot->getColumns(get_column_options)), storage_snapshot); - auto dummy_table_node = std::make_shared(std::move(storage_dummy), context); + auto dummy_table_node = std::make_shared(std::move(storage_dummy), getContext()); dummy_table_node->setAlias(node->getAlias()); replacement_map.emplace(node.get(), std::move(dummy_table_node)); } } - ContextPtr context; std::unordered_map replacement_map; }; QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) { - ReplaceTableNodeToDummyVisitor visitor; - visitor.context = context; + ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); return query->cloneAndReplace(visitor.replacement_map); diff --git a/src/Planner/findQueryForParallelReplicas.h b/src/Planner/findQueryForParallelReplicas.h index f5dc69dfa0e..cdce4ad0b47 100644 --- a/src/Planner/findQueryForParallelReplicas.h +++ b/src/Planner/findQueryForParallelReplicas.h @@ -13,7 +13,7 @@ using QueryTreeNodePtr = std::shared_ptr; struct SelectQueryOptions; -/// Find a qury which can be executed with parallel replicas up to WithMergableStage. +/// Find a query which can be executed with parallel replicas up to WithMergableStage. /// Returned query will always contain some (>1) subqueries, possibly with joins. const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options); diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql new file mode 100644 index 00000000000..12d2bd627a7 --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -0,0 +1,15 @@ +create table t (number UInt64) engine MergeTree order by number; + +SELECT 1 +FROM +( + SELECT number IN ( + SELECT number + FROM view( + SELECT number + FROM numbers(1) + ) + ) + FROM t +) +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; From 7160e954c16100e963371e416878837437569d74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 13:32:24 +0200 Subject: [PATCH 265/496] 02995_new_settings_history: Update ref --- ..._23_12_1.tsv => 02995_baseline_24_7_1.tsv} | 182 +++++++++++++++--- .../0_stateless/02995_new_settings_history.sh | 14 +- 2 files changed, 167 insertions(+), 29 deletions(-) rename tests/queries/0_stateless/{02995_baseline_23_12_1.tsv => 02995_baseline_24_7_1.tsv} (82%) diff --git a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv b/tests/queries/0_stateless/02995_baseline_24_7_1.tsv similarity index 82% rename from tests/queries/0_stateless/02995_baseline_23_12_1.tsv rename to tests/queries/0_stateless/02995_baseline_24_7_1.tsv index a391473e7c9..6c830da8646 100644 --- a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_1.tsv @@ -11,23 +11,28 @@ allow_create_index_without_type 0 allow_custom_error_code_in_throwif 0 allow_ddl 1 allow_deprecated_database_ordinary 0 +allow_deprecated_error_prone_window_functions 0 +allow_deprecated_snowflake_conversion_functions 0 allow_deprecated_syntax_for_merge_tree 0 allow_distributed_ddl 1 allow_drop_detached 0 allow_execute_multiif_columnar 1 allow_experimental_alter_materialized_view_structure 1 -allow_experimental_analyzer 0 +allow_experimental_analyzer 1 allow_experimental_annoy_index 0 allow_experimental_bigint_types 1 allow_experimental_codecs 0 allow_experimental_database_atomic 1 allow_experimental_database_materialized_mysql 0 allow_experimental_database_materialized_postgresql 0 -allow_experimental_database_replicated 0 +allow_experimental_database_replicated 1 +allow_experimental_dynamic_type 0 +allow_experimental_full_text_index 0 allow_experimental_funnel_functions 0 allow_experimental_geo_types 1 allow_experimental_hash_functions 0 allow_experimental_inverted_index 0 +allow_experimental_join_condition 0 allow_experimental_lightweight_delete 1 allow_experimental_live_view 0 allow_experimental_map_type 1 @@ -40,12 +45,15 @@ allow_experimental_query_cache 1 allow_experimental_query_deduplication 0 allow_experimental_refreshable_materialized_view 0 allow_experimental_s3queue 1 -allow_experimental_shared_merge_tree 0 +allow_experimental_shared_merge_tree 1 +allow_experimental_statistic 0 allow_experimental_statistics 0 allow_experimental_undrop_table_query 1 allow_experimental_usearch_index 0 +allow_experimental_variant_type 0 allow_experimental_window_functions 1 allow_experimental_window_view 0 +allow_get_client_http_header 0 allow_hyperscan 1 allow_introspection_functions 0 allow_named_collection_override_by_default 1 @@ -58,17 +66,21 @@ allow_prefetched_read_pool_for_remote_filesystem 1 allow_push_predicate_when_subquery_contains_with 1 allow_settings_after_format_in_insert 0 allow_simdjson 1 +allow_statistic_optimize 0 allow_statistics_optimize 0 allow_suspicious_codecs 0 allow_suspicious_fixed_string_types 0 allow_suspicious_indices 0 allow_suspicious_low_cardinality_types 0 +allow_suspicious_primary_key 0 allow_suspicious_ttl_expressions 0 +allow_suspicious_variant_types 0 allow_unrestricted_reads_from_keeper 0 alter_move_to_space_execute_async 0 alter_partition_verbose_result 0 alter_sync 1 analyze_index_with_space_filling_curves 1 +analyzer_compatibility_join_using_top_level_identifier 0 annoy_index_search_k_nodes -1 any_join_distinct_right_table_keys 0 apply_deleted_mask 1 @@ -76,20 +88,42 @@ apply_mutations_on_fly 0 asterisk_include_alias_columns 0 asterisk_include_materialized_columns 0 async_insert 0 +async_insert_busy_timeout_decrease_rate 0.2 +async_insert_busy_timeout_increase_rate 0.2 +async_insert_busy_timeout_max_ms 200 +async_insert_busy_timeout_min_ms 50 async_insert_busy_timeout_ms 200 async_insert_cleanup_timeout_ms 1000 async_insert_deduplicate 0 -async_insert_max_data_size 1000000 +async_insert_max_data_size 10485760 async_insert_max_query_number 450 +async_insert_poll_timeout_ms 10 async_insert_stale_timeout_ms 0 async_insert_threads 16 +async_insert_use_adaptive_busy_timeout 1 async_query_sending_for_remote 1 async_socket_for_remote 1 +azure_allow_parallel_part_upload 1 azure_create_new_file_on_insert 0 +azure_ignore_file_doesnt_exist 0 azure_list_object_keys_size 1000 +azure_max_blocks_in_multipart_upload 50000 +azure_max_inflight_parts_for_one_file 20 +azure_max_single_part_copy_size 268435456 azure_max_single_part_upload_size 104857600 azure_max_single_read_retries 4 +azure_max_unexpected_write_error_retries 4 +azure_max_upload_part_size 5368709120 +azure_min_upload_part_size 16777216 +azure_sdk_max_retries 10 +azure_sdk_retry_initial_backoff_ms 10 +azure_sdk_retry_max_backoff_ms 1000 +azure_skip_empty_files 0 +azure_strict_upload_part_size 0 +azure_throw_on_zero_files_match 0 azure_truncate_on_insert 0 +azure_upload_part_size_multiply_factor 2 +azure_upload_part_size_multiply_parts_count_threshold 500 background_buffer_flush_schedule_pool_size 16 background_common_pool_size 8 background_distributed_schedule_pool_size 16 @@ -107,6 +141,7 @@ backup_restore_keeper_max_retries 20 backup_restore_keeper_retry_initial_backoff_ms 100 backup_restore_keeper_retry_max_backoff_ms 5000 backup_restore_keeper_value_max_size 1048576 +backup_restore_s3_retry_attempts 1000 backup_threads 16 bool_false_representation false bool_true_representation true @@ -115,6 +150,7 @@ calculate_text_stack_trace 1 cancel_http_readonly_queries_on_client_close 0 cast_ipv4_ipv6_default_on_conversion_error 0 cast_keep_nullable 0 +cast_string_to_dynamic_use_inference 0 check_query_single_value_result 1 check_referential_table_dependencies 0 check_table_dependencies 1 @@ -123,6 +159,7 @@ cloud_mode 0 cloud_mode_engine 1 cluster_for_parallel_replicas collect_hash_table_stats_during_aggregation 1 +collect_hash_table_stats_during_joins 1 column_names_for_schema_inference compatibility compatibility_ignore_auto_increment_in_create_table 0 @@ -141,9 +178,12 @@ count_distinct_optimization 0 create_index_ignore_unique 0 create_replicated_merge_tree_fault_injection_probability 0 create_table_empty_primary_key_by_default 0 +cross_join_min_bytes_to_compress 1073741824 +cross_join_min_rows_to_compress 10000000 cross_to_inner_join_rewrite 1 data_type_default_nullable 0 database_atomic_wait_for_drop_and_detach_synchronously 0 +database_replicated_allow_heavy_create 0 database_replicated_allow_only_replicated_engine 0 database_replicated_allow_replicated_engine_arguments 1 database_replicated_always_detach_permanently 0 @@ -156,15 +196,19 @@ date_time_overflow_behavior ignore decimal_check_overflow 1 deduplicate_blocks_in_dependent_materialized_views 0 default_database_engine Atomic +default_materialized_view_sql_security DEFINER default_max_bytes_in_join 1000000000 -default_table_engine None +default_normal_view_sql_security INVOKER +default_table_engine MergeTree default_temporary_table_engine Memory +default_view_definer CURRENT_USER describe_compact_output 0 describe_extend_object_types 0 describe_include_subcolumns 0 describe_include_virtual_columns 0 dialect clickhouse dictionary_use_async_executor 0 +dictionary_validate_primary_key_type 0 distinct_overflow_mode throw distributed_aggregation_memory_efficient 1 distributed_background_insert_batch 0 @@ -182,6 +226,7 @@ distributed_directory_monitor_sleep_time_ms 100 distributed_directory_monitor_split_batch_on_failure 0 distributed_foreground_insert 0 distributed_group_by_no_merge 0 +distributed_insert_skip_read_only_replicas 0 distributed_product_mode deny distributed_push_down_limit 1 distributed_replica_error_cap 1000 @@ -191,6 +236,7 @@ do_not_merge_across_partitions_select_final 0 drain_timeout 3 empty_result_for_aggregation_by_constant_keys_on_empty_set 1 empty_result_for_aggregation_by_empty_set 0 +enable_blob_storage_log 1 enable_debug_queries 0 enable_deflate_qpl_codec 0 enable_early_constant_folding 1 @@ -205,6 +251,7 @@ enable_job_stack_trace 0 enable_lightweight_delete 1 enable_memory_bound_merging_of_aggregation_results 1 enable_multiple_prewhere_read_steps 1 +enable_named_columns_in_function_tuple 1 enable_optimize_predicate_expression 1 enable_optimize_predicate_expression_to_final_subquery 1 enable_order_by_all 1 @@ -216,7 +263,9 @@ enable_sharing_sets_for_mutations 1 enable_software_prefetch_in_aggregation 1 enable_unaligned_array_join 0 enable_url_encoding 1 +enable_vertical_final 1 enable_writes_to_query_cache 1 +enable_zstd_qat_codec 0 engine_file_allow_create_multiple_files 0 engine_file_empty_if_not_exists 0 engine_file_skip_empty_files 0 @@ -231,10 +280,12 @@ external_storage_max_read_rows 0 external_storage_rw_timeout_sec 300 external_table_functions_use_nulls 1 external_table_strict_query 0 +extract_key_value_pairs_max_pairs_per_row 1000 extract_kvp_max_pairs_per_row 1000 extremes 0 fallback_to_stale_replicas_for_distributed_queries 1 filesystem_cache_max_download_size 137438953472 +filesystem_cache_reserve_space_wait_lock_timeout_milliseconds 1000 filesystem_cache_segments_batch_size 20 filesystem_prefetch_max_memory_usage 1073741824 filesystem_prefetch_min_bytes_for_single_read_task 2097152 @@ -278,7 +329,9 @@ format_regexp_escaping_rule Raw format_regexp_skip_unmatched 0 format_schema format_template_resultset +format_template_resultset_format format_template_row +format_template_row_format format_template_rows_between_delimiter \n format_tsv_null_representation \\N formatdatetime_f_prints_single_zero 0 @@ -288,8 +341,11 @@ fsync_metadata 1 function_implementation function_json_value_return_type_allow_complex 0 function_json_value_return_type_allow_nullable 0 +function_locate_has_mysql_compatible_argument_order 1 function_range_max_elements_in_block 500000000 function_sleep_max_microseconds_per_block 3000000 +function_visible_width_behavior 1 +geo_distance_returns_float64_on_float64_arguments 1 glob_expansion_max_elements 1000 grace_hash_join_initial_buckets 1 grace_hash_join_max_buckets 1024 @@ -300,8 +356,10 @@ group_by_use_nulls 0 handle_kafka_error_mode default handshake_timeout_ms 10000 hdfs_create_new_file_on_insert 0 +hdfs_ignore_file_doesnt_exist 0 hdfs_replication 0 hdfs_skip_empty_files 0 +hdfs_throw_on_zero_files_match 0 hdfs_truncate_on_insert 0 hedged_connection_timeout_ms 50 hsts_max_age 0 @@ -326,10 +384,14 @@ http_skip_not_found_url_for_globs 1 http_wait_end_of_query 0 http_write_exception_in_output_format 1 http_zlib_compression_level 3 +iceberg_engine_ignore_schema_evolution 0 idle_connection_timeout 3600 ignore_cold_parts_seconds 0 ignore_data_skipping_indices +ignore_drop_queries_probability 0 +ignore_materialized_views_with_dropped_target_table 0 ignore_on_cluster_for_replicated_access_entities_queries 0 +ignore_on_cluster_for_replicated_named_collections_queries 0 ignore_on_cluster_for_replicated_udf_queries 0 implicit_transaction 0 input_format_allow_errors_num 0 @@ -341,12 +403,14 @@ input_format_arrow_import_nested 0 input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_avro_allow_missing_fields 0 input_format_avro_null_as_default 0 +input_format_binary_decode_types_in_binary_format 0 input_format_bson_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_csv_allow_cr_end_of_line 0 input_format_csv_allow_variable_number_of_columns 0 input_format_csv_allow_whitespace_or_tab_as_delimiter 0 input_format_csv_arrays_as_nested_csv 0 +input_format_csv_deserialize_separate_columns_into_tuple 1 input_format_csv_detect_header 1 input_format_csv_empty_as_default 1 input_format_csv_enum_as_number 0 @@ -354,29 +418,37 @@ input_format_csv_skip_first_lines 0 input_format_csv_skip_trailing_empty_lines 0 input_format_csv_trim_whitespaces 1 input_format_csv_try_infer_numbers_from_strings 0 +input_format_csv_try_infer_strings_from_quoted_tuples 1 input_format_csv_use_best_effort_in_schema_inference 1 input_format_csv_use_default_on_bad_values 0 input_format_custom_allow_variable_number_of_columns 0 input_format_custom_detect_header 1 input_format_custom_skip_trailing_empty_lines 0 input_format_defaults_for_omitted_fields 1 +input_format_force_null_for_omitted_fields 0 +input_format_hive_text_allow_variable_number_of_columns 1 input_format_hive_text_collection_items_delimiter  input_format_hive_text_fields_delimiter  input_format_hive_text_map_keys_delimiter  input_format_import_nested_json 0 input_format_ipv4_default_on_conversion_error 0 input_format_ipv6_default_on_conversion_error 0 +input_format_json_case_insensitive_column_matching 0 input_format_json_compact_allow_variable_number_of_columns 0 input_format_json_defaults_for_missing_elements_in_named_tuple 1 input_format_json_ignore_unknown_keys_in_named_tuple 1 +input_format_json_ignore_unnecessary_fields 1 input_format_json_infer_incomplete_types_as_strings 1 input_format_json_named_tuples_as_objects 1 input_format_json_read_arrays_as_strings 1 input_format_json_read_bools_as_numbers 1 +input_format_json_read_bools_as_strings 1 input_format_json_read_numbers_as_strings 1 input_format_json_read_objects_as_strings 1 +input_format_json_throw_on_bad_escape_sequence 1 input_format_json_try_infer_named_tuples_from_objects 1 input_format_json_try_infer_numbers_from_strings 0 +input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects 0 input_format_json_validate_types_from_metadata 1 input_format_max_bytes_to_read_for_schema_inference 33554432 input_format_max_rows_to_read_for_schema_inference 25000 @@ -384,11 +456,13 @@ input_format_msgpack_number_of_columns 0 input_format_mysql_dump_map_column_names 1 input_format_mysql_dump_table_name input_format_native_allow_types_conversion 1 +input_format_native_decode_types_in_binary_format 0 input_format_null_as_default 1 input_format_orc_allow_missing_columns 1 input_format_orc_case_insensitive_column_matching 0 input_format_orc_filter_push_down 1 input_format_orc_import_nested 0 +input_format_orc_read_use_writer_time_zone 0 input_format_orc_row_batch_size 100000 input_format_orc_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_orc_use_fast_decoder 1 @@ -398,17 +472,21 @@ input_format_parquet_case_insensitive_column_matching 0 input_format_parquet_filter_push_down 1 input_format_parquet_import_nested 0 input_format_parquet_local_file_min_bytes_for_seek 8192 -input_format_parquet_max_block_size 8192 +input_format_parquet_max_block_size 65409 +input_format_parquet_prefer_block_bytes 16744704 input_format_parquet_preserve_order 0 input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference 0 +input_format_parquet_use_native_reader 0 input_format_protobuf_flatten_google_wrappers 0 input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_record_errors_file_path input_format_skip_unknown_fields 1 input_format_try_infer_dates 1 input_format_try_infer_datetimes 1 +input_format_try_infer_exponent_floats 0 input_format_try_infer_integers 1 input_format_tsv_allow_variable_number_of_columns 0 +input_format_tsv_crlf_end_of_line 0 input_format_tsv_detect_header 1 input_format_tsv_empty_as_default 0 input_format_tsv_enum_as_number 0 @@ -450,7 +528,12 @@ joined_subquery_requires_alias 1 kafka_disable_num_consumers_limit 0 kafka_max_wait_ms 5000 keeper_map_strict_mode 0 +keeper_max_retries 10 +keeper_retry_initial_backoff_ms 100 +keeper_retry_max_backoff_ms 5000 legacy_column_name_of_tuple_literal 0 +lightweight_deletes_sync 2 +lightweight_mutation_projection_mode throw limit 0 live_view_heartbeat_interval 15 load_balancing random @@ -461,7 +544,7 @@ local_filesystem_read_prefetch 0 lock_acquire_timeout 120 log_comment log_formatted_queries 0 -log_processors_profiles 0 +log_processors_profiles 1 log_profile_events 1 log_queries 1 log_queries_cut_to_length 100000 @@ -474,6 +557,8 @@ log_query_views 1 low_cardinality_allow_in_native_format 1 low_cardinality_max_dictionary_size 8192 low_cardinality_use_single_dictionary_for_part 0 +materialize_skip_indexes_on_insert 1 +materialize_statistics_on_insert 1 materialize_ttl_after_modify 1 materialized_views_ignore_errors 0 max_alter_threads \'auto(16)\' @@ -501,6 +586,7 @@ max_distributed_depth 5 max_download_buffer_size 10485760 max_download_threads 4 max_entries_for_hash_table_stats 10000 +max_estimated_execution_time 0 max_execution_speed 0 max_execution_speed_bytes 0 max_execution_time 0 @@ -528,7 +614,9 @@ max_network_bandwidth_for_user 0 max_network_bytes 0 max_number_of_partitions_for_independent_aggregation 128 max_parallel_replicas 1 +max_parser_backtracks 1000000 max_parser_depth 1000 +max_parsing_threads \'auto(16)\' max_partition_size_to_drop 50000000000 max_partitions_per_insert_block 100 max_partitions_to_read -1 @@ -537,6 +625,7 @@ max_query_size 262144 max_read_buffer_size 1048576 max_read_buffer_size_local_fs 131072 max_read_buffer_size_remote_fs 0 +max_recursive_cte_evaluation_depth 1000 max_remote_read_network_bandwidth 0 max_remote_read_network_bandwidth_for_server 0 max_remote_write_network_bandwidth 0 @@ -549,7 +638,7 @@ max_result_rows 0 max_rows_in_distinct 0 max_rows_in_join 0 max_rows_in_set 0 -max_rows_in_set_to_optimize_join 100000 +max_rows_in_set_to_optimize_join 0 max_rows_to_group_by 0 max_rows_to_read 0 max_rows_to_read_leaf 0 @@ -557,6 +646,7 @@ max_rows_to_sort 0 max_rows_to_transfer 0 max_sessions_for_user 0 max_size_to_preallocate_for_aggregation 100000000 +max_size_to_preallocate_for_joins 100000000 max_streams_for_merge_tree_reading 0 max_streams_multiplier_for_merge_tables 5 max_streams_to_max_threads_ratio 1 @@ -592,6 +682,7 @@ merge_tree_min_bytes_per_task_for_remote_reading 4194304 merge_tree_min_rows_for_concurrent_read 163840 merge_tree_min_rows_for_concurrent_read_for_remote_filesystem 163840 merge_tree_min_rows_for_seek 0 +merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0 merge_tree_use_const_size_tasks_for_remote_reading 1 metrics_perf_events_enabled 0 metrics_perf_events_list @@ -604,6 +695,8 @@ min_count_to_compile_expression 3 min_count_to_compile_sort_description 3 min_execution_speed 0 min_execution_speed_bytes 0 +min_external_table_block_size_bytes 268402944 +min_external_table_block_size_rows 1048449 min_free_disk_space_for_temporary_data 0 min_hit_rate_to_use_consecutive_keys_optimization 0.5 min_insert_block_size_bytes 268402944 @@ -619,8 +712,8 @@ mutations_execute_subqueries_on_initiator 0 mutations_max_literal_size_to_replace 16384 mutations_sync 0 mysql_datatypes_support_level -mysql_map_fixed_string_to_text_in_show_columns 0 -mysql_map_string_to_text_in_show_columns 0 +mysql_map_fixed_string_to_text_in_show_columns 1 +mysql_map_string_to_text_in_show_columns 1 mysql_max_rows_to_insert 65536 network_compression_method LZ4 network_zstd_compression_level 1 @@ -647,6 +740,7 @@ optimize_group_by_constant_keys 1 optimize_group_by_function_keys 1 optimize_if_chain_to_multiif 0 optimize_if_transform_strings_to_enum 0 +optimize_injective_functions_in_group_by 1 optimize_injective_functions_inside_uniq 1 optimize_min_equality_disjunction_chain_length 3 optimize_min_inequality_conjunction_chain_length 3 @@ -664,7 +758,7 @@ optimize_redundant_functions_in_order_by 1 optimize_respect_aliases 1 optimize_rewrite_aggregate_function_with_if 1 optimize_rewrite_array_exists_to_has 0 -optimize_rewrite_sum_if_to_count_if 0 +optimize_rewrite_sum_if_to_count_if 1 optimize_skip_merged_partitions 0 optimize_skip_unused_shards 0 optimize_skip_unused_shards_limit 1000 @@ -674,9 +768,10 @@ optimize_sorting_by_input_stream_properties 1 optimize_substitute_columns 0 optimize_syntax_fuse_functions 0 optimize_throw_if_noop 0 +optimize_time_filter_with_preimage 1 optimize_trivial_approximate_count_query 0 optimize_trivial_count_query 1 -optimize_trivial_insert_select 1 +optimize_trivial_insert_select 0 optimize_uniq_to_count 1 optimize_use_implicit_projections 1 optimize_use_projections 1 @@ -685,13 +780,19 @@ os_thread_priority 0 output_format_arrow_compression_method lz4_frame output_format_arrow_fixed_string_as_fixed_byte_array 1 output_format_arrow_low_cardinality_as_dictionary 0 -output_format_arrow_string_as_string 0 +output_format_arrow_string_as_string 1 +output_format_arrow_use_64_bit_indexes_for_dictionary 0 +output_format_arrow_use_signed_indexes_for_dictionary 1 output_format_avro_codec output_format_avro_rows_in_file 1 output_format_avro_string_column_pattern output_format_avro_sync_interval 16384 +output_format_binary_encode_types_in_binary_format 0 output_format_bson_string_as_string 0 +output_format_compression_level 3 +output_format_compression_zstd_window_log 0 output_format_csv_crlf_end_of_line 0 +output_format_csv_serialize_tuple_into_separate_columns 1 output_format_decimal_trailing_zeros 0 output_format_enable_streaming 0 output_format_json_array_of_rows 0 @@ -705,27 +806,34 @@ output_format_json_skip_null_value_in_named_tuples 0 output_format_json_validate_utf8 0 output_format_markdown_escape_special_characters 0 output_format_msgpack_uuid_representation ext -output_format_orc_compression_method lz4 +output_format_native_encode_types_in_binary_format 0 +output_format_orc_compression_method zstd output_format_orc_row_index_stride 10000 -output_format_orc_string_as_string 0 +output_format_orc_string_as_string 1 output_format_parallel_formatting 1 output_format_parquet_batch_size 1024 output_format_parquet_compliant_nested_types 1 -output_format_parquet_compression_method lz4 +output_format_parquet_compression_method zstd output_format_parquet_data_page_size 1048576 output_format_parquet_fixed_string_as_fixed_byte_array 1 output_format_parquet_parallel_encoding 1 output_format_parquet_row_group_size 1000000 output_format_parquet_row_group_size_bytes 536870912 -output_format_parquet_string_as_string 0 -output_format_parquet_use_custom_encoder 0 +output_format_parquet_string_as_string 1 +output_format_parquet_use_custom_encoder 1 output_format_parquet_version 2.latest -output_format_pretty_color 1 +output_format_parquet_write_page_index 1 +output_format_pretty_color auto +output_format_pretty_display_footer_column_names 1 +output_format_pretty_display_footer_column_names_min_rows 50 output_format_pretty_grid_charset UTF-8 +output_format_pretty_highlight_digit_groups 1 output_format_pretty_max_column_pad_width 250 output_format_pretty_max_rows 10000 output_format_pretty_max_value_width 10000 -output_format_pretty_row_numbers 0 +output_format_pretty_max_value_width_apply_for_single_value 0 +output_format_pretty_row_numbers 1 +output_format_pretty_single_large_number_tip_threshold 1000000 output_format_protobuf_nullables_with_google_wrappers 0 output_format_schema output_format_sql_insert_include_column_names 1 @@ -734,15 +842,22 @@ output_format_sql_insert_quote_names 1 output_format_sql_insert_table_name table output_format_sql_insert_use_replace 0 output_format_tsv_crlf_end_of_line 0 +output_format_values_escape_quote_with_quote 0 output_format_write_statistics 1 +page_cache_inject_eviction 0 parallel_distributed_insert_select 0 parallel_replica_offset 0 +parallel_replicas_allow_in_with_subquery 1 parallel_replicas_count 0 parallel_replicas_custom_key parallel_replicas_custom_key_filter_type default +parallel_replicas_custom_key_range_lower 0 +parallel_replicas_custom_key_range_upper 0 parallel_replicas_for_non_replicated_merge_tree 0 +parallel_replicas_mark_segment_size 128 parallel_replicas_min_number_of_granules_to_enable 0 parallel_replicas_min_number_of_rows_per_replica 0 +parallel_replicas_prefer_local_join 1 parallel_replicas_single_task_marks_count_multiplier 2 parallel_view_processing 0 parallelize_output_from_storages 1 @@ -755,11 +870,14 @@ parts_to_delay_insert 0 parts_to_throw_insert 0 periodic_live_view_refresh 60 poll_interval 10 +postgresql_connection_attempt_timeout 2 postgresql_connection_pool_auto_close_connection 0 +postgresql_connection_pool_retries 2 postgresql_connection_pool_size 16 postgresql_connection_pool_wait_timeout 5000 precise_float_parsing 0 prefer_column_name_to_alias 0 +prefer_external_sort_block_bytes 16744704 prefer_global_in_and_join 0 prefer_localhost_replica 1 prefer_warmed_unmerged_parts_seconds 0 @@ -767,7 +885,7 @@ preferred_block_size_bytes 1000000 preferred_max_column_in_block_size_bytes 0 preferred_optimize_projection_name prefetch_buffer_size 1048576 -print_pretty_type_names 0 +print_pretty_type_names 1 priority 0 query_cache_compress_entries 1 query_cache_max_entries 0 @@ -778,8 +896,10 @@ query_cache_nondeterministic_function_handling throw query_cache_share_between_users 0 query_cache_squash_partial_results 1 query_cache_store_results_of_queries_with_nondeterministic_functions 0 +query_cache_system_table_handling throw query_cache_ttl 60 query_plan_aggregation_in_order 1 +query_plan_convert_outer_join_to_inner_join 1 query_plan_enable_multithreading_after_window_functions 1 query_plan_enable_optimizations 1 query_plan_execute_functions_after_sorting 1 @@ -788,6 +908,8 @@ query_plan_lift_up_array_join 1 query_plan_lift_up_union 1 query_plan_max_optimizations_to_apply 10000 query_plan_merge_expressions 1 +query_plan_merge_filters 0 +query_plan_optimize_prewhere 1 query_plan_optimize_primary_key 1 query_plan_optimize_projection 1 query_plan_push_down_limit 1 @@ -806,7 +928,9 @@ read_backoff_min_events 2 read_backoff_min_interval_between_events_ms 1000 read_backoff_min_latency_ms 1000 read_from_filesystem_cache_if_exists_otherwise_bypass_cache 0 +read_from_page_cache_if_exists_otherwise_bypass_cache 0 read_in_order_two_level_merge_threshold 100 +read_in_order_use_buffering 1 read_overflow_mode throw read_overflow_mode_leaf throw read_priority 0 @@ -835,17 +959,20 @@ result_overflow_mode throw rewrite_count_distinct_if_with_count_distinct_implementation 0 s3_allow_parallel_part_upload 1 s3_check_objects_after_upload 0 +s3_connect_timeout_ms 1000 s3_create_new_file_on_insert 0 s3_disable_checksum 0 -s3_http_connection_pool_size 1000 +s3_ignore_file_doesnt_exist 0 s3_list_object_keys_size 1000 s3_max_connections 1024 s3_max_get_burst 0 s3_max_get_rps 0 s3_max_inflight_parts_for_one_file 20 +s3_max_part_number 10000 s3_max_put_burst 0 s3_max_put_rps 0 s3_max_redirects 10 +s3_max_single_operation_copy_size 33554432 s3_max_single_part_upload_size 33554432 s3_max_single_read_retries 4 s3_max_unexpected_write_error_retries 4 @@ -860,6 +987,8 @@ s3_truncate_on_insert 0 s3_upload_part_size_multiply_factor 2 s3_upload_part_size_multiply_parts_count_threshold 500 s3_use_adaptive_timeouts 1 +s3_validate_request_settings 1 +s3queue_allow_experimental_sharded_mode 0 s3queue_default_zookeeper_path /clickhouse/s3queue/ s3queue_enable_logging_to_s3queue_log 0 schema_inference_cache_require_modification_time_for_url 1 @@ -887,6 +1016,8 @@ sleep_after_receiving_query_ms 0 sleep_in_send_data_ms 0 sleep_in_send_tables_status_ms 0 sort_overflow_mode throw +split_intersecting_parts_ranges_into_layers_final 1 +split_parts_ranges_into_intersecting_and_non_intersecting_final 1 splitby_max_substrings_includes_remaining_string 0 stop_refreshable_materialized_views_on_startup 0 storage_file_read_method pread @@ -898,8 +1029,10 @@ stream_poll_timeout_ms 500 system_events_show_zero_values 0 table_function_remote_max_addresses 1000 tcp_keep_alive_timeout 290 +temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds 600000 temporary_files_codec LZ4 temporary_live_view_timeout 1 +throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert 1 throw_if_no_data_to_insert 1 throw_on_error_from_cache_on_write_operations 0 throw_on_max_partitions_per_insert_block 1 @@ -912,8 +1045,10 @@ totals_mode after_having_exclusive trace_profile_events 0 transfer_overflow_mode throw transform_null_in 0 +traverse_shadow_remote_data_paths 0 union_default_mode unknown_packet_in_send_data 0 +update_insert_deduplication_token_in_dependent_materialized_views 0 use_cache_for_count_from_files 1 use_client_time_zone 0 use_compact_format_in_distributed_parts_names 1 @@ -923,12 +1058,15 @@ use_index_for_in_with_subqueries 1 use_index_for_in_with_subqueries_max_values 0 use_local_cache_for_remote_storage 1 use_mysql_types_in_show_columns 0 +use_page_cache_for_disks_without_file_cache 0 use_query_cache 0 use_skip_indexes 1 use_skip_indexes_if_final 0 use_structure_from_insertion_table_in_table_functions 2 use_uncompressed_cache 0 +use_variant_as_common_type 0 use_with_fill_by_sorting_prefix 1 +validate_experimental_and_suspicious_types_inside_nested_types 1 validate_polygons 1 wait_changes_become_visible_after_commit_mode wait_unknown wait_for_async_insert 1 diff --git a/tests/queries/0_stateless/02995_new_settings_history.sh b/tests/queries/0_stateless/02995_new_settings_history.sh index 8de98c55b6a..917dacc04b0 100755 --- a/tests/queries/0_stateless/02995_new_settings_history.sh +++ b/tests/queries/0_stateless/02995_new_settings_history.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # Note that this is a broad check. A per version check is done in the upgrade test -# Baseline generated with 23.12.1 -# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_23_12_1.tsv +# Baseline generated with 24.7.1 +# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_1.tsv $CLICKHOUSE_LOCAL --query " WITH old_settings AS ( - SELECT * FROM file('${CUR_DIR}/02995_baseline_23_12_1.tsv', 'TSV', 'name String, default String') + SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_1.tsv', 'TSV', 'name String, default String') ), new_settings AS ( @@ -21,7 +21,7 @@ $CLICKHOUSE_LOCAL --query " ) SELECT * FROM ( - SELECT 'PLEASE ADD THE NEW SETTING TO SettingsChangesHistory.h: ' || name || ' WAS ADDED', + SELECT 'PLEASE ADD THE NEW SETTING TO SettingsChangesHistory.cpp: ' || name || ' WAS ADDED', FROM new_settings WHERE (name NOT IN ( SELECT name @@ -29,17 +29,17 @@ $CLICKHOUSE_LOCAL --query " )) AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM system.settings_changes - WHERE splitByChar('.', version())[1] >= '24' + WHERE splitByChar('.', version)[1]::UInt64 >= 24 AND splitByChar('.', version)[2]::UInt64 > 7 )) UNION ALL ( - SELECT 'PLEASE ADD THE SETTING VALUE CHANGE TO SettingsChangesHistory.h: ' || name || ' WAS CHANGED FROM ' || old_settings.default || ' TO ' || new_settings.default, + SELECT 'PLEASE ADD THE SETTING VALUE CHANGE TO SettingsChangesHistory.cpp: ' || name || ' WAS CHANGED FROM ' || old_settings.default || ' TO ' || new_settings.default, FROM new_settings LEFT JOIN old_settings ON new_settings.name = old_settings.name WHERE (new_settings.default != old_settings.default) AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM system.settings_changes - WHERE splitByChar('.', version())[1] >= '24' + WHERE splitByChar('.', version)[1]::UInt64 >= 24 AND splitByChar('.', version)[2]::UInt64 > 7 )) ) ) From b178eea09ec80fed40b5043ccf1635d95b9cf19b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 14:59:17 +0200 Subject: [PATCH 266/496] Fix broken settings --- src/Core/SettingsChangesHistory.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..ecc558e64d7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,16 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.12",{ + }}, + {"24.11",{ + }}, + {"24.10",{ + }}, + {"24.9", { + }}, + {"24.8", {{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, + }}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, @@ -76,7 +86,6 @@ static std::initializer_list Date: Wed, 31 Jul 2024 15:11:55 +0200 Subject: [PATCH 267/496] Try a less conflict prone format --- src/Core/SettingsChangesHistory.cpp | 709 +++++++++++++++++----------- 1 file changed, 441 insertions(+), 268 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ecc558e64d7..21c89b3c5c5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,274 +57,447 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.12",{ - }}, - {"24.11",{ - }}, - {"24.10",{ - }}, - {"24.9", { - }}, - {"24.8", {{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - }}, - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, + {"24.12", + { + } + }, + {"24.11", + { + } + }, + {"24.10", + { + } + }, + {"24.9", + { + } + }, + {"24.8", + { + {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, + } + }, + {"24.7", + { + {"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, + {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, + {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, + {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, + {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, + {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, + {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, + {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, + {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, + {"collect_hash_table_stats_during_joins", false, true, "New setting."}, + {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, + {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, + {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, + {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, + {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, + {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, + {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, + {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, + {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, + {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, + {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, + {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} + } + }, + {"24.6", + { + {"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, + {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, + {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, + {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, + {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, + {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, + {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, + {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, + {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, + {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, + {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, + {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, + {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, + {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, + {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, + {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, + {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, + {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, + {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, + {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, + {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, + {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, + {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, + {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, + {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, + {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, + {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, + {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, + } + }, + {"24.5", + { + {"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, + {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, + {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, + {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, + {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, + {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, + {"http_max_chunk_size", 0, 0, "Internal limitation"}, + {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, + {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, + {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, + {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, + {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, + } + }, + {"24.4", + { + {"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, + {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, + {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, + {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, + {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, + {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, + {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, + {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, + {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, + {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, + {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, + {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, + {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, + } + }, + {"24.3", + { + {"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, + {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, + {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, + {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, + {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, + {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, + {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, + {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, + {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, + {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, + {"log_processors_profiles", false, true, "Enable by default"}, + {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, + {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, + {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, + {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, + {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, + {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, + {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, + {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, + {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, + {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, + {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, + {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, + {"allow_get_client_http_header", false, false, "Introduced a new function."}, + {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, + {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, + {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, + {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, + {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, + {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, + {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, + {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, + {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, + {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, + {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, + {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, + {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, + {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, + {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, + {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, + {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, + {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, + } + }, + {"24.2", + { + {"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, + {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, + {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, + {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, + {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, + {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, + {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, + {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, + {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, + {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, + {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, + {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, + {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, + {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, + {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, + {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, + {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, + {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, + {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, + {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, + {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, + {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, + {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, + {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, + {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, + {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, + {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + } + }, + {"24.1", + { + {"print_pretty_type_names", false, true, "Better user experience."}, + {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, + {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, + {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, + {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, + {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, + {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, + {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, + {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, + {"enable_vertical_final", false, true, "Use vertical final by default"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, + {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, + {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, + {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, + {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, + {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, + {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, + {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, + {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"} + } + }, + {"23.12", + { + {"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, + {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, + {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, + {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"} + } + }, + {"23.11", + { + {"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"} + } + }, + {"23.9", + { + {"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, + {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, + {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, + {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, + {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, + {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, + {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."} + } + }, + {"23.8", + { + {"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"} + } + }, + {"23.7", + { + {"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."} + } + }, + {"23.6", + { + {"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, + {"http_receive_timeout", 180, 30, "See http_send_timeout."} + } + }, + {"23.5", + { + {"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, + {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, + {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."} + } + }, + {"23.4", + { + {"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, + {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, + {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, + {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, + {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, + {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, + {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"} + } + }, + {"23.3", + { + {"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, + {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, + {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, + {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, + {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, + {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, + {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"} + } + }, + {"23.2", + { + {"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, + {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, + {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, + {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, + {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"} + } + }, + {"23.1", + { + {"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, + {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, + {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, + {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, + {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, + {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"} + } + }, + {"22.12", + { + {"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, + {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, + {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"} + } + }, + {"22.11", + { + {"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"} + } + }, + {"22.9", + { + {"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"} + } + }, + {"22.7", + { + {"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, + {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"} + } + }, + {"22.6", + { + {"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, + {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"} + } + }, + {"22.5", + { + {"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, + {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"} + } + }, + {"22.4", + { + {"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"} + } + }, + {"22.3", + { + {"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"} + } + }, + {"21.12", + { + {"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"} + } + }, + {"21.9", + { + {"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"} + } + }, + {"21.7", + { + {"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"} + } + }, + {"21.5", + { + {"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"} + } + }, + {"21.3", + { + {"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, + {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, + {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"} + } + }, + {"21.2", + { + {"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"} + } + }, + {"21.1", + { + {"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, + {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, + {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, + {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"} + } + }, + {"20.10", + { + {"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"} + } + }, + {"20.7", + { + {"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"} + } + }, + {"20.5", + { + {"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, + {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"} + } + }, + {"20.4", + { + {"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"} + } + }, + {"19.18", + { + {"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"} + } + }, + {"19.14", + { + {"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"} + } + }, + {"19.12", + { + {"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"} + } + }, + {"19.5", + { + {"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"} + } + }, + {"18.12.17", + { + {"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"} + } + }, }; From bca21bc6c735b710ed15061e41790066b5ffd843 Mon Sep 17 00:00:00 2001 From: sakulali Date: Wed, 31 Jul 2024 21:33:14 +0800 Subject: [PATCH 268/496] add test cases --- src/Common/Config/ConfigProcessor.cpp | 11 ++++------- ...89_clickhouse_client_config_format.reference | 4 ++++ .../01889_clickhouse_client_config_format.sh | 17 +++++++++++++++++ 3 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index f0b83c035c8..c4b4a1d5e7e 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -654,22 +654,19 @@ XMLDocumentPtr ConfigProcessor::parseConfig(const std::string & config_path) while (std::getline(file, line)) { const size_t pos = firstNonWhitespacePos(line); - if (pos == std::string::npos) - continue; if (pos < line.size() && '<' == line[pos]) { maybe_xml = true; + break; } - - break; + else if (pos != std::string::npos) + break; } } - if (maybe_xml) return dom_parser.parse(config_path); - else - return YAMLParser::parse(config_path); + return YAMLParser::parse(config_path); } } diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index afe27ddb063..ef0d9ffc538 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -18,3 +18,7 @@ autodetect xml (non leading whitespaces) 2 autodetect yaml 2 +autodetect invalid xml +Code: 1000, e.code() = 0, SAXParseException: Invalid token in '/config_test.badxml', line 2 column 12, Stack trace (when copying this message, always include the lines below): +autodetect invalid yaml +Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 14d67f943f1..9a44ec0d5f5 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -15,6 +15,8 @@ yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml autodetect_xml_with_leading_whitespace_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.config autodetect_xml_non_leading_whitespace_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.cfg autodetect_yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.properties +autodetect_invalid_xml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.badxml +autodetect_invalid_yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.badyaml function cleanup() { @@ -27,6 +29,8 @@ function cleanup() rm "${autodetect_xml_with_leading_whitespace_config:?}" rm "${autodetect_xml_non_leading_whitespace_config:?}" rm "${autodetect_yaml_config:?}" + rm "${autodetect_invalid_xml_config:?}" + rm "${autodetect_invalid_yaml_config:?}" } trap cleanup EXIT @@ -70,6 +74,15 @@ EOL cat > "$autodetect_yaml_config" < "$autodetect_invalid_xml_config" < + +EOL +cat > "$autodetect_invalid_yaml_config" <&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#Poco::Exception. ##" +echo 'autodetect invalid yaml' +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file From 31c142a96d49fbe1b46b21e4cdad366546dc7864 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:44:54 +0100 Subject: [PATCH 269/496] make it possible to rerun test_storage_delta and test_checking_s3_blobs_paranoid --- .../test_checking_s3_blobs_paranoid/test.py | 2 ++ tests/integration/test_storage_delta/test.py | 29 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index dde636b5d29..afe8449b44a 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -61,6 +61,7 @@ def test_upload_after_check_works(cluster, broken_s3): node.query( """ + DROP TABLE IF EXISTS s3_upload_after_check_works; CREATE TABLE s3_upload_after_check_works ( id Int64, data String @@ -631,6 +632,7 @@ def test_no_key_found_disk(cluster, broken_s3): node.query( """ + DROP TABLE IF EXISTS no_key_found_disk; CREATE TABLE no_key_found_disk ( id Int64 ) ENGINE=MergeTree() diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 67cc7cdd6da..698becc18c4 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -52,6 +52,11 @@ def get_spark(): return builder.master("local").getOrCreate() +def remove_local_directory_contents(local_path): + for local_file in glob.glob(local_path + "/**"): + os.unlink(local_file) + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -169,6 +174,9 @@ def test_single_log_file(started_cluster): inserted_data ) + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_partition_by(started_cluster): instance = started_cluster.instances["node1"] @@ -191,6 +199,7 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 + remove_local_directory_contents(f"/{TABLE_NAME}") def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] @@ -266,6 +275,9 @@ def test_checkpoint(started_cluster): ).strip() ) + remove_local_directory_contents(f"/{TABLE_NAME}") + spark.sql(f"DROP TABLE {TABLE_NAME}") + def test_multiple_log_files(started_cluster): instance = started_cluster.instances["node1"] @@ -304,6 +316,8 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_metadata(started_cluster): instance = started_cluster.instances["node1"] @@ -337,6 +351,9 @@ def test_metadata(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_types(started_cluster): TABLE_NAME = "test_types" @@ -409,6 +426,9 @@ def test_types(started_cluster): ] ) + remove_local_directory_contents(f"/{result_file}") + spark.sql(f"DROP TABLE {TABLE_NAME}") + def test_restart_broken(started_cluster): instance = started_cluster.instances["node1"] @@ -470,6 +490,9 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_restart_broken_table_function(started_cluster): instance = started_cluster.instances["node1"] @@ -524,6 +547,9 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_partition_columns(started_cluster): instance = started_cluster.instances["node1"] @@ -721,3 +747,6 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) + + remove_local_directory_contents(f"/{TABLE_NAME}") + spark.sql(f"DROP TABLE {TABLE_NAME}") From 2a2dba63cc0182247754a5a4819cb89f21825bfd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:48:04 +0000 Subject: [PATCH 270/496] Automatic style fix --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From 7dbd3d75340522195e7d08a725cf5ae116288c8e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:51:38 +0000 Subject: [PATCH 271/496] Automatic style fix --- tests/integration/test_storage_delta/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 698becc18c4..e485bc90ee0 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -201,6 +201,7 @@ def test_partition_by(started_cluster): remove_local_directory_contents(f"/{TABLE_NAME}") + def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session From 27f4f468b976e445e8b0dbc198ea9f0a9c62855b Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:55:00 +0100 Subject: [PATCH 272/496] make it possible to rerun test_recovery_time_metric multiple times --- tests/integration/test_recovery_time_metric/test.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 628f2e744e0..6fcf2fad423 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -21,6 +21,7 @@ def start_cluster(): def test_recovery_time_metric(start_cluster): node.query( """ + DROP DATABASE IF EXISTS rdb; CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') """ @@ -28,6 +29,7 @@ def test_recovery_time_metric(start_cluster): node.query( """ + DROP TABLE IF EXISTS rdb.t; CREATE TABLE rdb.t ( `x` UInt32 @@ -51,3 +53,9 @@ def test_recovery_time_metric(start_cluster): ).strip() ) assert ret > 0 + + node.query( + """ + DROP DATABASE rdb + """ + ) From d6de2be4395e1bcc62ab32ad1d5b02e9db080303 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:08:18 +0200 Subject: [PATCH 273/496] Fix build --- src/Planner/findParallelReplicasQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 1140f30ad9c..39edb1e6516 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -144,7 +144,7 @@ public: std::unordered_map replacement_map; }; -QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) +QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const ContextPtr & context) { ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); From 9ffbd8f5073e180592a494742d1dc3af4427b55f Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:13:43 +0000 Subject: [PATCH 274/496] Possible fix --- .../03164_s3_settings_for_queries_and_merges.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index ac2070fbd76..e43c9ae7717 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -22,18 +22,18 @@ SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; SELECT - ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE event_date >= yesterday() AND type = 'QueryFinish' +WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; SELECT - ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE event_date >= yesterday() AND type = 'QueryFinish' +WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%OPTIMIZE TABLE t_compact_bytes_s3 FINAL%'; From 67f4792b77f2a2cf0de21ead6e95c3635d26aa88 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:16:40 +0000 Subject: [PATCH 275/496] Style check --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From 1f1f0528ce3a1fb20ceee5513523787a14718b80 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 31 Jul 2024 14:32:07 +0000 Subject: [PATCH 276/496] Prefer constant to INPUT in PlannerActionsVisitor. --- src/Planner/PlannerActionsVisitor.cpp | 11 +++++++- ...lyzer_materialized_constants_bug.reference | 3 +++ ...15_analyzer_materialized_constants_bug.sql | 26 +++++++++++++++++++ 3 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference create mode 100644 tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 1960855792c..57457493844 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -491,7 +491,16 @@ public: { auto it = node_name_to_node.find(node_name); if (it != node_name_to_node.end()) - return it->second; + { + /// It is possible that ActionsDAG already has an input with the same name as constant. + /// In this case, prefer constant to input. + /// Constatns affect function return type, which should be consistent with QueryTree. + /// Query example: + /// SELECT materialize(toLowCardinality('b')) || 'a' FROM remote('127.0.0.{1,2}', system, one) GROUP BY 'a' + bool materialized_input = it->second->type == ActionsDAG::ActionType::INPUT && !it->second->column; + if (!materialized_input) + return it->second; + } const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference new file mode 100644 index 00000000000..584e34c0cde --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference @@ -0,0 +1,3 @@ +ba +\N +1 111111111111111111111111111111111111111 diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql new file mode 100644 index 00000000000..f9ec28d09d8 --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql @@ -0,0 +1,26 @@ +SET allow_experimental_analyzer = 1; + +SELECT concat(materialize(toLowCardinality('b')), 'a') FROM remote('127.0.0.{1,2}', system, one) GROUP BY 'a'; + +SELECT concat(NULLIF(1, materialize(toLowCardinality(1))), concat(NULLIF(1, 1))) FROM remote('127.0.0.{1,2}', system, one) GROUP BY concat(NULLIF(1, 1)); + +DROP TABLE IF EXISTS test__fuzz_21; +CREATE TABLE test__fuzz_21 +( + `x` Decimal(18, 10) +) +ENGINE = MergeTree +ORDER BY x; + +INSERT INTO test__fuzz_21 VALUES (1), (2), (3); + +WITH ( + SELECT CAST(toFixedString(toFixedString(materialize(toFixedString('111111111111111111111111111111111111111', 39)), 39), 39), 'UInt128') + ) AS v +SELECT + coalesce(materialize(toLowCardinality(toNullable(1))), 10, NULL), + max(v) +FROM remote('127.0.0.{1,2}', default, test__fuzz_21) +GROUP BY + coalesce(NULL), + coalesce(1, 10, 10, materialize(NULL)); From e31569a065d4c81cdea671727c39983d7f3a84e5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:32:37 +0200 Subject: [PATCH 277/496] Expect an unknown cluster --- .../0_stateless/03215_analyzer_replace_with_dummy_tables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql index 12d2bd627a7..6d084c2ac50 100644 --- a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -12,4 +12,4 @@ FROM ) FROM t ) -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; -- { serverError CLUSTER_DOESNT_EXIST } From 8c36fbf4eddeba9282b53f726976b55f62d3ee19 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:42:38 +0000 Subject: [PATCH 278/496] Remove unnecessary change --- access/quotas.list | Bin 0 -> 1 bytes access/roles.list | Bin 0 -> 1 bytes access/row_policies.list | Bin 0 -> 1 bytes access/settings_profiles.list | Bin 0 -> 1 bytes access/users.list | Bin 0 -> 1 bytes .../03164_s3_settings_for_queries_and_merges.sql | 4 ++-- 6 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 access/quotas.list create mode 100644 access/roles.list create mode 100644 access/row_policies.list create mode 100644 access/settings_profiles.list create mode 100644 access/users.list diff --git a/access/quotas.list b/access/quotas.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/roles.list b/access/roles.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/row_policies.list b/access/row_policies.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/settings_profiles.list b/access/settings_profiles.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/users.list b/access/users.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index e43c9ae7717..94e390537df 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -25,7 +25,7 @@ SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE type = 'QueryFinish' +WHERE event_date >= yesterday() AND type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; @@ -33,7 +33,7 @@ SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE type = 'QueryFinish' +WHERE event_date >= yesterday() AND type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%OPTIMIZE TABLE t_compact_bytes_s3 FINAL%'; From 20ec27f9dc79d7ee81cd06f1587de83c8ce81441 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:46:53 +0000 Subject: [PATCH 279/496] Remove trach dir --- access/quotas.list | Bin 1 -> 0 bytes access/roles.list | Bin 1 -> 0 bytes access/row_policies.list | Bin 1 -> 0 bytes access/settings_profiles.list | Bin 1 -> 0 bytes access/users.list | Bin 1 -> 0 bytes 5 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 access/quotas.list delete mode 100644 access/roles.list delete mode 100644 access/row_policies.list delete mode 100644 access/settings_profiles.list delete mode 100644 access/users.list diff --git a/access/quotas.list b/access/quotas.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/roles.list b/access/roles.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/row_policies.list b/access/row_policies.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/settings_profiles.list b/access/settings_profiles.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/users.list b/access/users.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 From 650737890299f8cad2c77ad46022ee0a37b284eb Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:49:30 +0000 Subject: [PATCH 280/496] Fix erroe with profile event name --- .../0_stateless/03164_s3_settings_for_queries_and_merges.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index 94e390537df..001ef382850 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -22,7 +22,7 @@ SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; SELECT - ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' @@ -30,7 +30,7 @@ WHERE event_date >= yesterday() AND type = 'QueryFinish' AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; SELECT - ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' From c3c653e7692a755c3467b77e866555734d50ef50 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 Jul 2024 14:58:52 +0000 Subject: [PATCH 281/496] Better --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ .../config.d/database_catalog_drop_table_concurrency.xml | 3 +++ tests/config/install.sh | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/config/config.d/database_catalog_drop_table_concurrency.xml diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index a8e5fd7e6aa..48b01a9df43 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,6 +1406,8 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids).contains(uuid) ? "table still exists" : "table dropped successfully"); + /// TSA doesn't support unique_lock if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " diff --git a/tests/config/config.d/database_catalog_drop_table_concurrency.xml b/tests/config/config.d/database_catalog_drop_table_concurrency.xml new file mode 100644 index 00000000000..ac118625f4e --- /dev/null +++ b/tests/config/config.d/database_catalog_drop_table_concurrency.xml @@ -0,0 +1,3 @@ + + 256 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 1b0edc5fc16..7c4b36dc4bd 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -21,6 +21,7 @@ ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ From 5ab8c0357a84d0265bdb43f43795b06317854772 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Jul 2024 16:30:14 +0200 Subject: [PATCH 282/496] Fix potential busy loop in keepFreeSpaceRatioFunc --- src/Interpreters/Cache/FileCache.cpp | 30 +++++++++++++------ src/Interpreters/Cache/IFileCachePriority.h | 8 ++++- .../Cache/LRUFileCachePriority.cpp | 21 +++++++++++-- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- .../Cache/SLRUFileCachePriority.cpp | 15 ++++++---- .../Cache/SLRUFileCachePriority.h | 2 +- 6 files changed, 57 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index a88c0de2cfe..bf8dd24a1db 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -998,18 +998,19 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() FileCacheReserveStat stat; EvictionCandidates eviction_candidates; - bool limits_satisfied = true; + IFileCachePriority::DesiredSizeStatus desired_size_status; try { /// Collect at most `keep_up_free_space_remove_batch` elements to evict, /// (we use batches to make sure we do not block cache for too long, /// by default the batch size is quite small). - limits_satisfied = main_priority->collectCandidatesForEviction( + desired_size_status = main_priority->collectCandidatesForEviction( desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, eviction_candidates, lock); #ifdef DEBUG_OR_SANITIZER_BUILD /// Let's make sure that we correctly processed the limits. - if (limits_satisfied && eviction_candidates.size() < keep_up_free_space_remove_batch) + if (desired_size_status == IFileCachePriority::DesiredSizeStatus::SUCCESS + && eviction_candidates.size() < keep_up_free_space_remove_batch) { const auto current_size = main_priority->getSize(lock); chassert(current_size >= stat.total_stat.releasable_size); @@ -1063,13 +1064,24 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() watch.stop(); ProfileEvents::increment(ProfileEvents::FilesystemCacheFreeSpaceKeepingThreadWorkMilliseconds, watch.elapsedMilliseconds()); - LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds()); + LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms (status: {})", + watch.elapsedMilliseconds(), desired_size_status); [[maybe_unused]] bool scheduled = false; - if (limits_satisfied) - scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - else - scheduled = keep_up_free_space_ratio_task->schedule(); + switch (desired_size_status) + { + case IFileCachePriority::DesiredSizeStatus::SUCCESS: [[fallthrough]]; + case IFileCachePriority::DesiredSizeStatus::CANNOT_EVICT: + { + scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + break; + } + case IFileCachePriority::DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT: + { + scheduled = keep_up_free_space_ratio_task->schedule(); + break; + } + } chassert(scheduled); } @@ -1546,7 +1558,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheReserveStat stat; if (main_priority->collectCandidatesForEviction( new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, - stat, eviction_candidates, cache_lock)) + stat, eviction_candidates, cache_lock) == IFileCachePriority::DesiredSizeStatus::SUCCESS) { if (eviction_candidates.size() == 0) { diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 5d8eb9dd54a..9885ab00f78 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -151,7 +151,13 @@ public: /// and `desired_elements_num` as current cache state. /// Collect no more than `max_candidates_to_evict` elements. /// Return `true` if the first condition is satisfied. - virtual bool collectCandidatesForEviction( + enum class DesiredSizeStatus + { + SUCCESS, + CANNOT_EVICT, + REACHED_MAX_CANDIDATES_LIMIT, + }; + virtual DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index ec96eb14a8a..7970eaa3e13 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -323,7 +323,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -bool LRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -336,12 +336,24 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock, &desired_size, &desired_elements_count); }; + auto status = DesiredSizeStatus::CANNOT_EVICT; auto stop_condition = [&]() { - return desired_limits_satisfied() || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); + if (desired_limits_satisfied()) + { + status = DesiredSizeStatus::SUCCESS; + return true; + } + if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) + { + status = DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT; + return true; + } + return false; }; iterateForEviction(res, stat, stop_condition, lock); - return desired_limits_satisfied(); + chassert(status != DesiredSizeStatus::SUCCESS || stop_condition()); + return status; } void LRUFileCachePriority::iterateForEviction( @@ -350,6 +362,9 @@ void LRUFileCachePriority::iterateForEviction( StopConditionFunc stop_condition, const CachePriorityGuard::Lock & lock) { + if (stop_condition()) + return; + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); IterateFunc iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index e0691cade43..9bced106727 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -63,7 +63,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - bool collectCandidatesForEviction( + DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 7a3fdf5160e..dc0df223cb0 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -256,7 +256,7 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -bool SLRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -268,7 +268,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); FileCacheReserveStat probationary_stat; - const bool probationary_limit_satisfied = probationary_queue.collectCandidatesForEviction( + const auto probationary_desired_size_status = probationary_queue.collectCandidatesForEviction( desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, probationary_stat, res, lock); @@ -285,14 +285,14 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) - return probationary_limit_satisfied; + if (probationary_desired_size_status == DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT) + return probationary_desired_size_status; const auto desired_protected_size = getRatio(desired_size, size_ratio); const auto desired_protected_elements_num = getRatio(desired_elements_count, size_ratio); FileCacheReserveStat protected_stat; - const bool protected_limit_satisfied = protected_queue.collectCandidatesForEviction( + const auto protected_desired_size_status = protected_queue.collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, max_candidates_to_evict - res.size(), protected_stat, res, lock); @@ -306,7 +306,10 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, protected_queue.getStateInfoForLog(lock)); - return probationary_limit_satisfied && protected_limit_satisfied; + if (probationary_desired_size_status == DesiredSizeStatus::SUCCESS) + return protected_desired_size_status; + else + return probationary_desired_size_status; } void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 2102a0ec558..e6d20e0d0ee 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,7 +58,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - bool collectCandidatesForEviction( + DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, From 19cd00000373a5707178214744444b4d8c4034a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Jul 2024 17:18:55 +0200 Subject: [PATCH 283/496] Update src/Interpreters/DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 48b01a9df43..56d9c323d39 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,7 +1406,7 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); - LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids).contains(uuid) ? "table still exists" : "table dropped successfully"); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids.contains(uuid) ? "table still exists" : "table dropped successfully"); /// TSA doesn't support unique_lock if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) From f032c015ca9ec10b7938bbf3d67bb6181776d24a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 15:40:37 +0000 Subject: [PATCH 284/496] Ignore some tests --- tests/queries/0_stateless/00705_drop_create_merge_tree.sh | 4 ++-- .../0_stateless/01019_alter_materialized_view_atomic.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index fd002668696..ea8b9d02e49 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT & -yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT & +yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & +yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & wait ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table" diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index eb12a76eb62..4bd21fcee02 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT < Date: Wed, 31 Jul 2024 15:41:37 +0000 Subject: [PATCH 285/496] Fix build --- src/Interpreters/DatabaseCatalog.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 56d9c323d39..273e5720679 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,10 +1406,11 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); - LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids.contains(uuid) ? "table still exists" : "table dropped successfully"); - /// TSA doesn't support unique_lock - if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) + const bool has_table = TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), has_table ? "table still exists" : "table dropped successfully"); + + if has_table) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " "will finish after restart", uuid); } From 8afe61e04581d0b95ac2d6e927bb9d2427247c7a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 Jul 2024 17:58:41 +0200 Subject: [PATCH 286/496] Better --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 273e5720679..fb4fad85f66 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1410,7 +1410,7 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) const bool has_table = TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid); LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), has_table ? "table still exists" : "table dropped successfully"); - if has_table) + if (has_table) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " "will finish after restart", uuid); } From 743d63767a74f41b3628c52ccf166be773baecf2 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 31 Jul 2024 17:06:49 +0000 Subject: [PATCH 287/496] fix AsyncLoader destruction race --- src/Common/AsyncLoader.cpp | 33 ++++++++++++++++++++------------- 1 file changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6264eb03106..d40e320e741 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -218,20 +218,27 @@ AsyncLoader::~AsyncLoader() { // All `LoadTask` objects should be destructed before AsyncLoader destruction because they hold a reference. // To make sure we check for all pending jobs to be finished. - std::unique_lock lock{mutex}; - if (scheduled_jobs.empty() && finished_jobs.empty()) - return; + { + std::unique_lock lock{mutex}; + if (!scheduled_jobs.empty() || !finished_jobs.empty()) + { + std::vector scheduled; + std::vector finished; + scheduled.reserve(scheduled_jobs.size()); + finished.reserve(finished_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + scheduled.push_back(job->name); + for (const auto & job : finished_jobs) + finished.push_back(job->name); + LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); + abort(); + } + } - std::vector scheduled; - std::vector finished; - scheduled.reserve(scheduled_jobs.size()); - finished.reserve(finished_jobs.size()); - for (const auto & [job, _] : scheduled_jobs) - scheduled.push_back(job->name); - for (const auto & job : finished_jobs) - finished.push_back(job->name); - LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); - abort(); + // When all jobs are done we could still have finalizing workers. + // These workers could call updateCurrentPriorityAndSpawn() that scans all pools. + // We need to stop all of them before destructing any of them. + stop(); } void AsyncLoader::start() From 2b79da36c0701bb9ca392fddd9129a7e0e04ef3f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 19:26:45 +0200 Subject: [PATCH 288/496] Update 01605_adaptive_granularity_block_borders.sql --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 5f09dc423b2..f9b8bb1c1c6 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache +-- Tags: long, no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher From bfb0133f26f85e64f2d608595b653c20a594abca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 289/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 2c6c5c5c94a2f4131d898b172d6af285eec07c2f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 19:31:22 +0200 Subject: [PATCH 290/496] Add an assertion --- src/Common/TimerDescriptor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 9a171ae9487..716dcee2747 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -110,6 +110,9 @@ void TimerDescriptor::drain() const throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot readlink for a timer_fd {}", timer_fd); LOG_TRACE(log, "Received EINTR while trying to drain a TimerDescriptor, fd {}: {}", timer_fd, std::string_view(link_path, link_path_length)); + + /// Check that it's actually a timerfd. + chassert(std::string_view(link_path, link_path_length).contains("timerfd")); continue; } From 26a81e366d722314bbda12d5232fa49117ef4498 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 31 Jul 2024 18:05:53 +0000 Subject: [PATCH 291/496] Fix the serialization of parameters for parameterized views Removed the call to convertFieldToString() and added datatype specific serialization code. Parameterized view substitution was broken for multiple datatypes when parameter value was a function or expression returning datatype instance. Testcase added to cover Date/Date32/UUID/IP datatypes. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 10 +- ...zed_view_with_non_literal_params.reference | 31 ++++++ ...meterized_view_with_non_literal_params.sql | 97 +++++++++++++++++++ 3 files changed, 137 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference create mode 100644 tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..6113a38d463 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4546,7 +4546,15 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, resolveExpressionNode(nodes[1], scope, /* allow_lambda_expression */false, /* allow_table_function */false); if (auto * constant = nodes[1]->as()) { - view_params[identifier_node->getIdentifier().getFullName()] = convertFieldToString(constant->getValue()); + /// Serialize the constant value using datatype specific + /// interfaces to match the deserialization in ReplaceQueryParametersVistor. + WriteBufferFromOwnString buf; + auto constval = constant->getValue(); + auto realtype = constant->getResultType(); + auto tempcol = realtype->createColumn(); + tempcol->insert(constval); + realtype->getDefaultSerialization()->serializeTextEscaped(*tempcol, 0, buf, {}); + view_params[identifier_node->getIdentifier().getFullName()] = buf.str(); } } } diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference new file mode 100644 index 00000000000..e4e6c313b85 --- /dev/null +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference @@ -0,0 +1,31 @@ +Test with Date parameter +1 +2 +1 +3 +3 +3 +2 +Test with Date32 parameter +1 +2 +1 +3 +5 +3 +4 +Test with UUID parameter +4 +3 +3 +1 +2 +Test with 2 parameters +1 +1 +3 +3 +Test with IPv4 +1 +2 +3 diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql new file mode 100644 index 00000000000..55795c7a785 --- /dev/null +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -0,0 +1,97 @@ + +select 'Test with Date parameter'; + +drop table if exists date_table_pv; +create table date_table_pv (id Int32, dt Date) engine = Memory(); + +insert into date_table_pv values(1, today()); +insert into date_table_pv values(2, yesterday()); +insert into date_table_pv values(3, toDate('1974-04-07')); + +drop view if exists date_pv; +create view date_pv as select * from date_table_pv where dt = {dtparam:Date}; + +select id from date_pv(dtparam=today()); +select id from date_pv(dtparam=yesterday()); +select id from date_pv(dtparam=yesterday()+1); +select id from date_pv(dtparam='1974-04-07'); +select id from date_pv(dtparam=toDate('1974-04-07')); +select id from date_pv(dtparam=toString(toDate('1974-04-07'))); +select id from date_pv(dtparam=toDate('1975-04-07')); +select id from date_pv(dtparam=(select dt from date_table_pv where id = 2)); + +select 'Test with Date32 parameter'; + +drop table if exists date32_table_pv; +create table date32_table_pv (id Int32, dt Date32) engine = Memory(); + +insert into date32_table_pv values(1, today()); +insert into date32_table_pv values(2, yesterday()); +insert into date32_table_pv values(3, toDate32('2199-12-31')); +insert into date32_table_pv values(4, toDate32('1950-12-25')); +insert into date32_table_pv values(5, toDate32('1900-01-01')); + +drop view if exists date32_pv; +create view date32_pv as select * from date32_table_pv where dt = {dtparam:Date32}; + +select id from date32_pv(dtparam=today()); +select id from date32_pv(dtparam=yesterday()); +select id from date32_pv(dtparam=yesterday()+1); +select id from date32_pv(dtparam='2199-12-31'); +select id from date32_pv(dtparam=toDate32('1900-01-01')); +select id from date32_pv(dtparam=(select dt from date32_table_pv where id = 3)); +select id from date32_pv(dtparam=(select dt from date32_table_pv where id = 4)); + + +select 'Test with UUID parameter'; +drop table if exists uuid_table_pv; +create table uuid_table_pv (id Int32, uu UUID) engine = Memory(); + +insert into uuid_table_pv values(1, generateUUIDv4()); +insert into uuid_table_pv values(2, generateUUIDv7()); +insert into uuid_table_pv values(3, toUUID('11111111-2222-3333-4444-555555555555')); +insert into uuid_table_pv select 4, serverUUID(); + + +drop view if exists uuid_pv; +create view uuid_pv as select * from uuid_table_pv where uu = {uuidparam:UUID}; +select id from uuid_pv(uuidparam=serverUUID()); +select id from uuid_pv(uuidparam=toUUID('11111111-2222-3333-4444-555555555555')); +select id from uuid_pv(uuidparam='11111111-2222-3333-4444-555555555555'); +select id from uuid_pv(uuidparam=(select uu from uuid_table_pv where id = 1)); +select id from uuid_pv(uuidparam=(select uu from uuid_table_pv where id = 2)); +-- generateUUIDv4() is not constant foldable, hence cannot be used as parameter value +select id from uuid_pv(uuidparam=generateUUIDv4()); -- { serverError UNKNOWN_QUERY_PARAMETER } +-- But nested "select generateUUIDv4()" works! +select id from uuid_pv(uuidparam=(select generateUUIDv4())); + +select 'Test with 2 parameters'; + +drop view if exists date_pv2; +create view date_pv2 as select * from date_table_pv where dt = {dtparam:Date} and id = {intparam:Int32}; +select id from date_pv2(dtparam=today(),intparam=1); +select id from date_pv2(dtparam=today(),intparam=length('A')); +select id from date_pv2(dtparam='1974-04-07',intparam=length('AAA')); +select id from date_pv2(dtparam=toDate('1974-04-07'),intparam=length('BBB')); + +select 'Test with IPv4'; + +drop table if exists ipv4_table_pv; +create table ipv4_table_pv (id Int32, ipaddr IPv4) ENGINE = Memory(); +insert into ipv4_table_pv values (1, '116.106.34.242'); +insert into ipv4_table_pv values (2, '116.106.34.243'); +insert into ipv4_table_pv values (3, '116.106.34.244'); + +drop view if exists ipv4_pv; +create view ipv4_pv as select * from ipv4_table_pv where ipaddr = {ipv4param:IPv4}; +select id from ipv4_pv(ipv4param='116.106.34.242'); +select id from ipv4_pv(ipv4param=toIPv4('116.106.34.243')); +select id from ipv4_pv(ipv4param=(select ipaddr from ipv4_table_pv where id=3)); + +drop view date_pv; +drop view date_pv2; +drop view uuid_pv; +drop view ipv4_pv; +drop table date_table_pv; +drop table uuid_table_pv; +drop table ipv4_table_pv; From dde274f6fad979aa94ea31395b0434c81f72328a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 18:08:14 +0000 Subject: [PATCH 292/496] Re-enable ICU on s390/x --- contrib/icu-cmake/CMakeLists.txt | 4 +--- contrib/icudata | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index f9d05f7fe97..adeaa7dcf33 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -4,9 +4,7 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they generated -# the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255 -if (NOT ENABLE_ICU OR ARCH_S390X) +if (NOT ENABLE_ICU) message(STATUS "Not using ICU") return() endif() diff --git a/contrib/icudata b/contrib/icudata index d345d6ac22f..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit d345d6ac22f381c882420de9053d30ae1ff38d75 +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b From e2af1766eb1ea0ee2f6b862f53a0d3c13f53365b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:25:28 +0200 Subject: [PATCH 293/496] init --- src/Functions/DateTimeTransforms.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a7bd398cdaa..fe26c5cf353 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -381,11 +381,13 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + return res >= 0 ? res : 0; } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + return res >= 0 ? res : 0; } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { From d0c643180f408d84a5f10a917f413248b9267202 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:29:36 +0200 Subject: [PATCH 294/496] add tests --- .../03215_toStartOfWeek_with_dateTime64_fix.reference | 2 ++ .../0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference create mode 100644 tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference new file mode 100644 index 00000000000..fd698107f22 --- /dev/null +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference @@ -0,0 +1,2 @@ +1970-01-01 +1970-01-01 diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql new file mode 100644 index 00000000000..0f00a52cb86 --- /dev/null +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql @@ -0,0 +1,2 @@ +SELECT toStartOfWeek(toDateTime64('1970-02-01', 6)); +SELECT toStartOfWeek(toDateTime('1970-01-01')); From 636c3f642340de6e5ca4892481ca156cb236a4cd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:31:22 +0200 Subject: [PATCH 295/496] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index fe26c5cf353..1970ec3bdb0 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -382,12 +382,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - return res >= 0 ? res : 0; + return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - return res >= 0 ? res : 0; + return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { From 34ca3128ed0de60e03a105c43dc0924541f4a2c1 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:36:13 +0200 Subject: [PATCH 296/496] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 1970ec3bdb0..46fb3bb9f57 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -381,12 +381,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) From 9def8cea8121ae8001649629a96e73eb1e10159b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 18:57:08 +0000 Subject: [PATCH 297/496] Update version_date.tsv and changelogs after v24.4.4.107-stable --- docs/changelogs/v24.4.4.107-stable.md | 70 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 71 insertions(+) create mode 100644 docs/changelogs/v24.4.4.107-stable.md diff --git a/docs/changelogs/v24.4.4.107-stable.md b/docs/changelogs/v24.4.4.107-stable.md new file mode 100644 index 00000000000..ba7c576715e --- /dev/null +++ b/docs/changelogs/v24.4.4.107-stable.md @@ -0,0 +1,70 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.4.107-stable (af0ed6b197e) FIXME as compared to v24.4.3.25-stable (a915dd4eda4) + +#### Improvement +* Backported in [#65884](https://github.com/ClickHouse/ClickHouse/issues/65884): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65303](https://github.com/ClickHouse/ClickHouse/issues/65303): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65894](https://github.com/ClickHouse/ClickHouse/issues/65894): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65372](https://github.com/ClickHouse/ClickHouse/issues/65372): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#66883](https://github.com/ClickHouse/ClickHouse/issues/66883): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65435](https://github.com/ClickHouse/ClickHouse/issues/65435): Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65448](https://github.com/ClickHouse/ClickHouse/issues/65448): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65710](https://github.com/ClickHouse/ClickHouse/issues/65710): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66689](https://github.com/ClickHouse/ClickHouse/issues/66689): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65353](https://github.com/ClickHouse/ClickHouse/issues/65353): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65060](https://github.com/ClickHouse/ClickHouse/issues/65060): Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.`. [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65329](https://github.com/ClickHouse/ClickHouse/issues/65329): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#64833](https://github.com/ClickHouse/ClickHouse/issues/64833): Fix bug which could lead to non-working TTLs with expressions. [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Backported in [#65086](https://github.com/ClickHouse/ClickHouse/issues/65086): Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65540](https://github.com/ClickHouse/ClickHouse/issues/65540): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65578](https://github.com/ClickHouse/ClickHouse/issues/65578): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65161](https://github.com/ClickHouse/ClickHouse/issues/65161): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65616](https://github.com/ClickHouse/ClickHouse/issues/65616): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65730](https://github.com/ClickHouse/ClickHouse/issues/65730): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65668](https://github.com/ClickHouse/ClickHouse/issues/65668): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65786](https://github.com/ClickHouse/ClickHouse/issues/65786): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65810](https://github.com/ClickHouse/ClickHouse/issues/65810): Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65931](https://github.com/ClickHouse/ClickHouse/issues/65931): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65826](https://github.com/ClickHouse/ClickHouse/issues/65826): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66299](https://github.com/ClickHouse/ClickHouse/issues/66299): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66326](https://github.com/ClickHouse/ClickHouse/issues/66326): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66153](https://github.com/ClickHouse/ClickHouse/issues/66153): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66459](https://github.com/ClickHouse/ClickHouse/issues/66459): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66224](https://github.com/ClickHouse/ClickHouse/issues/66224): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66267](https://github.com/ClickHouse/ClickHouse/issues/66267): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66678](https://github.com/ClickHouse/ClickHouse/issues/66678): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66603](https://github.com/ClickHouse/ClickHouse/issues/66603): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66358](https://github.com/ClickHouse/ClickHouse/issues/66358): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66971](https://github.com/ClickHouse/ClickHouse/issues/66971): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66968](https://github.com/ClickHouse/ClickHouse/issues/66968): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66719](https://github.com/ClickHouse/ClickHouse/issues/66719): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66950](https://github.com/ClickHouse/ClickHouse/issues/66950): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66947](https://github.com/ClickHouse/ClickHouse/issues/66947): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67195](https://github.com/ClickHouse/ClickHouse/issues/67195): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67377](https://github.com/ClickHouse/ClickHouse/issues/67377): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67240](https://github.com/ClickHouse/ClickHouse/issues/67240): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65410](https://github.com/ClickHouse/ClickHouse/issues/65410): Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65903](https://github.com/ClickHouse/ClickHouse/issues/65903): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66385](https://github.com/ClickHouse/ClickHouse/issues/66385): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66424](https://github.com/ClickHouse/ClickHouse/issues/66424): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66542](https://github.com/ClickHouse/ClickHouse/issues/66542): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66857](https://github.com/ClickHouse/ClickHouse/issues/66857): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66873](https://github.com/ClickHouse/ClickHouse/issues/66873): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67057](https://github.com/ClickHouse/ClickHouse/issues/67057): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66944](https://github.com/ClickHouse/ClickHouse/issues/66944): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67250](https://github.com/ClickHouse/ClickHouse/issues/67250): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67410](https://github.com/ClickHouse/ClickHouse/issues/67410): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 027b207d3ad..abd8f84ec74 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -5,6 +5,7 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 +v24.4.4.107-stable 2024-07-31 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From cd3c6c3ae807a321d94079087eb1fb29f4764549 Mon Sep 17 00:00:00 2001 From: sakulali Date: Thu, 1 Aug 2024 03:37:45 +0800 Subject: [PATCH 298/496] try to fix flaky test 01889_clickhouse_client_config_format --- .../01889_clickhouse_client_config_format.reference | 2 +- .../0_stateless/01889_clickhouse_client_config_format.sh | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index ef0d9ffc538..2575200e6fa 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -19,6 +19,6 @@ autodetect xml (non leading whitespaces) autodetect yaml 2 autodetect invalid xml -Code: 1000, e.code() = 0, SAXParseException: Invalid token in '/config_test.badxml', line 2 column 12, Stack trace (when copying this message, always include the lines below): +Correct: invalid xml parsed with exception autodetect invalid yaml Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 9a44ec0d5f5..58fd6852116 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -108,7 +108,9 @@ echo 'autodetect xml (non leading whitespaces)' $CLICKHOUSE_CLIENT --config "$autodetect_xml_non_leading_whitespace_config" -q "select getSetting('max_threads')" echo 'autodetect yaml' $CLICKHOUSE_CLIENT --config "$autodetect_yaml_config" -q "select getSetting('max_threads')" + +# Error code is 1000 (Poco::Exception). It is not ignored. echo 'autodetect invalid xml' -$CLICKHOUSE_CLIENT --config "$autodetect_invalid_xml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#Poco::Exception. ##" +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_xml_config" -q "select getSetting('max_threads')" 2>&1 |& grep -q "Code: 1000" && echo "Correct: invalid xml parsed with exception" || echo 'Fail: expected error code 1000 but got other' echo 'autodetect invalid yaml' -$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file From cc27c254abd4b6fd8f64b47e0bdf6195041bd5ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 299/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 867784d55c989943d0c79eb9179b01e878fabcbe Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 22:48:16 +0200 Subject: [PATCH 300/496] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 46fb3bb9f57..ce7da406e9a 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include From 9cb52bd1381ad3e0929062801df7c4b542cf1117 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 23:11:35 +0200 Subject: [PATCH 301/496] fix build --- src/Functions/DateTimeTransforms.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index ce7da406e9a..15f1b9580f3 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -25,7 +24,7 @@ namespace DB static constexpr auto millisecond_multiplier = 1'000; static constexpr auto microsecond_multiplier = 1'000'000; -static constexpr auto nanosecond_multiplier = 1'000'000'000; +static constexpr auto nanosecond_multiplier = 1'000'000'000; static constexpr FormatSettings::DateTimeOverflowBehavior default_date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore; @@ -382,12 +381,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const int res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const int res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) From 6403f3f545bee153ffaf4ce5bda6fcde33ef88d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 23:29:54 +0200 Subject: [PATCH 302/496] Miscellaneous --- src/Common/Epoll.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 49c86222cf0..ef7c6e143a0 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -19,7 +19,7 @@ Epoll::Epoll() : events_count(0) { epoll_fd = epoll_create1(0); if (epoll_fd == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot open epoll descriptor"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot open epoll descriptor"); } Epoll::Epoll(Epoll && other) noexcept : epoll_fd(other.epoll_fd), events_count(other.events_count.load()) @@ -47,7 +47,7 @@ void Epoll::add(int fd, void * ptr, uint32_t events) ++events_count; if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot add new descriptor to epoll"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot add new descriptor to epoll"); } void Epoll::remove(int fd) @@ -55,7 +55,7 @@ void Epoll::remove(int fd) --events_count; if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot remove descriptor from epoll"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot remove descriptor from epoll"); } size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout) const @@ -82,7 +82,7 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout continue; } else - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Error in epoll_wait"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Error in epoll_wait"); } else break; From 6e914ff6da67be1c1381ffed2d04b5758704baf3 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 31 Jul 2024 21:59:37 +0000 Subject: [PATCH 303/496] Update settings.md Removing duplicate header "## background_merges_mutations_scheduling_policy" --- docs/en/operations/server-configuration-parameters/settings.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 8278f8c8699..a1e3c292b04 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -103,8 +103,6 @@ Default: 2 The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. -## background_merges_mutations_scheduling_policy - Algorithm used to select next merge or mutation to be executed by background thread pool. Policy may be changed at runtime without server restart. Could be applied from the `default` profile for backward compatibility. From 0e36db543762cb146aa6c233d4536fd62c6101b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 22:09:04 +0000 Subject: [PATCH 304/496] Do not check the value of `num_messages_read` because it is not reliable librdkafka usually polls both messages on the first SELECT, but sometimes only one. If it polls only one message first, then it will read two messages before rebalancing at the second SELECT from the table. This means it usually reads a single message twice (thus num_messages_read = 4 is usually fine as 1 discarded message + 3 actually consumed message). But when only one message is read in the first SELECT, then 2 messages are discarded, thus num_messages_read will be 5 as 2 discarded message + 3 actually consumed messages. --- tests/integration/test_storage_kafka/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8393e88db88..37457e00701 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4771,7 +4771,7 @@ def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): assignments.current_offset, if(length(exceptions.time)>0, exceptions.time[1]::String, 'never') as last_exception_time_, if(length(exceptions.text)>0, exceptions.text[1], 'no exception') as last_exception_, - stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, + stable_timestamp(last_poll_time) as last_poll_time_, stable_timestamp(last_commit_time) as last_commit_time_, num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, num_rebalance_revocations, num_rebalance_assignments, is_currently_used FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; @@ -4791,7 +4791,6 @@ assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 4 last_commit_time_: now num_commits: 2 last_rebalance_time_: now @@ -4810,7 +4809,6 @@ assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 1 last_commit_time_: now num_commits: 1 last_rebalance_time_: never From 587d50380ad3d1cadc56bda0fa700e3441c16ab4 Mon Sep 17 00:00:00 2001 From: sakulali Date: Thu, 1 Aug 2024 09:12:00 +0800 Subject: [PATCH 305/496] fix clickhouse-test reference --- .../0_stateless/01889_clickhouse_client_config_format.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index 2575200e6fa..149315ad9d5 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -21,4 +21,4 @@ autodetect yaml autodetect invalid xml Correct: invalid xml parsed with exception autodetect invalid yaml -Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) +Code: 585. Unable to parse YAML configuration file /config_default.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) From f162d6bd5e03c6f717b4f45cf4c7ba6491aaa5fa Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Aug 2024 06:35:22 +0000 Subject: [PATCH 306/496] Update version_date.tsv and changelogs after v24.7.2.13-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.7.2.13-stable.md | 24 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 +- 5 files changed, 28 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.7.2.13-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index e99c86267f9..94603763572 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.1.2915" +ARG VERSION="24.7.2.13" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index fb562b911a3..f40118c7b06 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.1.2915" +ARG VERSION="24.7.2.13" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 51f4e6a0f40..032aa862e4a 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.7.1.2915" +ARG VERSION="24.7.2.13" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.7.2.13-stable.md b/docs/changelogs/v24.7.2.13-stable.md new file mode 100644 index 00000000000..4a2fb665116 --- /dev/null +++ b/docs/changelogs/v24.7.2.13-stable.md @@ -0,0 +1,24 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.2.13-stable (6e41f601b2f) FIXME as compared to v24.7.1.2915-stable (a37d2d43da7) + +#### Improvement +* Backported in [#67531](https://github.com/ClickHouse/ClickHouse/issues/67531): In pr : https://github.com/ClickHouse/ClickHouse/pull/66025, we introduce a settings `input_format_orc_read_use_writer_time_zone` to fix when read orc file, make the reader use writer timezone, not always use `GMT`. [#67175](https://github.com/ClickHouse/ClickHouse/pull/67175) ([kevinyhzou](https://github.com/KevinyhZou)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#67505](https://github.com/ClickHouse/ClickHouse/issues/67505): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67580](https://github.com/ClickHouse/ClickHouse/issues/67580): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67551](https://github.com/ClickHouse/ClickHouse/issues/67551): [Green CI] Fix test test_storage_s3_queue/test.py::test_max_set_age. [#67035](https://github.com/ClickHouse/ClickHouse/pull/67035) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67514](https://github.com/ClickHouse/ClickHouse/issues/67514): Split test 02967_parallel_replicas_join_algo_and_analyzer. [#67211](https://github.com/ClickHouse/ClickHouse/pull/67211) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67545](https://github.com/ClickHouse/ClickHouse/issues/67545): [Green CI] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown. [#67474](https://github.com/ClickHouse/ClickHouse/pull/67474) ([Alexey Katsman](https://github.com/alexkats)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index abd8f84ec74..b1391c2d781 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 @@ -5,7 +6,6 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 -v24.4.4.107-stable 2024-07-31 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From c2df527a32d640f52296ea7aefae177e22504082 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 08:42:54 +0200 Subject: [PATCH 307/496] Reduce fault rate --- .../test_keeper_map_retries/configs/fault_injection.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml index 145945c7c7c..0933b6b3031 100644 --- a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -1,6 +1,6 @@ - 0.05 - 0.05 + 0.005 + 0.005 From 5564489cca1c14e95e7c543e03c508849abaf079 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 15:31:54 +0800 Subject: [PATCH 308/496] change as request --- .../functions/tuple-map-functions.md | 18 +++++++++++--- src/Functions/map.cpp | 24 ++++++++++--------- .../0_stateless/01651_map_functions.reference | 4 ++++ .../0_stateless/01651_map_functions.sql | 7 +++++- 4 files changed, 38 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index db66188b1f5..d670ed42a2a 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -43,7 +43,7 @@ Result: ## mapFromArrays -Creates a map from an array of keys and an array of values. +Creates a map from an array or map of keys and an array or map of values. The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`. For example, instead of writing @@ -62,8 +62,8 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). -- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). +- `keys` — Array or map of keys to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). If `keys` is an array, we accept `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as its type as long as it doesn't contain NULL value. +- `values` - Array or map of values to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** @@ -99,6 +99,18 @@ Result: └───────────────────────────────────────────────────────┘ ``` +```sql +SELECT mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3]) +``` + +Result: + +``` +┌─mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])─┐ +│ {('a',1):1,('b',2):2,('c',3):3} │ +└───────────────────────────────────────────────────────┘ +``` + ## extractKeyValuePairs Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md). diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index a8e5f7ad90e..738c61164a3 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } namespace @@ -157,7 +158,7 @@ private: bool use_variant_as_common_type = false; }; -/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays +/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays or maps class FunctionMapFromArrays : public IFunction { public: @@ -181,13 +182,13 @@ public: getName(), arguments.size()); - auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + auto get_nested_type = [&](const DataTypePtr & type) { DataTypePtr nested; - if (const auto * array_type = checkAndGetDataType(type.get())) - nested = array_type->getNestedType(); - else if (const auto * map_type = checkAndGetDataType(type.get())) - nested = std::make_shared(map_type->getKeyValueTypes()); + if (const auto * type_as_array = checkAndGetDataType(type.get())) + nested = type_as_array->getNestedType(); + else if (const auto * type_as_map = checkAndGetDataType(type.get())) + nested = std::make_shared(type_as_map->getKeyValueTypes()); else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -201,8 +202,9 @@ public: auto key_type = get_nested_type(arguments[0]); auto value_type = get_nested_type(arguments[1]); - /// Remove Nullable from key_type if needed for map key must not be Nullable + /// We accept Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as key types as long as the actual array doesn't contain NULL value(this is checked in executeImpl). key_type = removeNullableOrLowCardinalityNullable(key_type); + DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -210,7 +212,7 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - auto get_array_column = [this](const ColumnPtr & column) -> std::pair + auto get_array_column = [&](const ColumnPtr & column) -> std::pair { bool is_const = isColumnConst(*column); ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; @@ -231,8 +233,9 @@ public: }; auto [col_keys, key_holder] = get_array_column(arguments[0].column); + auto [col_values, values_holder] = get_array_column(arguments[1].column); - /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + /// Nullable(T) or LowCardinality(Nullable(T)) are okay as nested key types but actual NULL values are not okay. ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { @@ -253,10 +256,9 @@ public: if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); + ErrorCodes::BAD_ARGUMENTS, "The nested column of first argument in function {} must not contain NULLs", getName()); } - auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 9114aa419b1..e336a02318d 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -55,3 +55,7 @@ {1:3,2:4} {1:3,2:4} {1:3,2:4} {(1,3):'a',(2,4):'b'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 4604ddd6db1..dc93a38b265 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -68,7 +68,7 @@ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAY select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError BAD_ARGUMENTS } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); @@ -79,3 +79,8 @@ select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, select mapFromArrays([1,2], [3,4]); select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); + +select mapFromArrays(map(1, 'a', 2, 'b'), array('c', 'd')); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), array('c', 'd')); +select mapFromArrays(map(1, 'a', 2, 'b'), materialize(array('c', 'd'))); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), materialize(array('c', 'd'))); From 681441e170202bc3963fb3fa1d7b7785192dbd2e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 16:01:39 +0800 Subject: [PATCH 309/496] fix style --- src/Functions/map.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 738c61164a3..6e389f39dec 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -192,7 +192,7 @@ public: else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument types of function {} must be Array or Map, but {} is given", + "Arguments of function {} must be Array or Map, but {} is given", getName(), type->getName()); @@ -275,10 +275,7 @@ public: static constexpr auto name = "mapUpdate"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -287,9 +284,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 2", - getName(), arguments.size()); + getName(), + arguments.size()); const auto * left = checkAndGetDataType(arguments[0].type.get()); const auto * right = checkAndGetDataType(arguments[1].type.get()); @@ -405,7 +404,6 @@ public: return ColumnMap::create(nested_column); } }; - } REGISTER_FUNCTION(Map) From 7db4065898633ace1f909711d4caeda8d135cace Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 10:30:50 +0200 Subject: [PATCH 310/496] Add retries to create --- src/Storages/StorageKeeperMap.cpp | 373 +++++++++++------- src/Storages/StorageKeeperMap.h | 6 +- .../configs/keeper_retries.xml | 14 + tests/integration/test_keeper_map/test.py | 3 +- .../configs/fault_injection.xml | 1 + .../configs/keeper_retries.xml | 14 + .../test_keeper_map_retries/test.py | 13 +- 7 files changed, 275 insertions(+), 149 deletions(-) create mode 100644 tests/integration/test_keeper_map/configs/keeper_retries.xml create mode 100644 tests/integration/test_keeper_map_retries/configs/keeper_retries.xml diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 0634c7be6ee..a6be9f8da04 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -408,104 +408,192 @@ StorageKeeperMap::StorageKeeperMap( if (attach) { - checkTable(); + checkTable(context_); return; } - auto client = getClient(); + const auto & settings = context_->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{settings.keeper_max_retries, settings.keeper_retry_initial_backoff_ms, settings.keeper_retry_max_backoff_ms}, + context_->getProcessListElement()}; - if (zk_root_path != "/" && !client->exists(zk_root_path)) - { - LOG_TRACE(log, "Creating root path {}", zk_root_path); - client->createAncestors(zk_root_path); - client->createIfNotExists(zk_root_path, ""); - } + zk_retry.retryLoop( + [&] + { + auto client = getClient(); + if (zk_root_path != "/" && !client->exists(zk_root_path)) + { + LOG_TRACE(log, "Creating root path {}", zk_root_path); + client->createAncestors(zk_root_path); + client->createIfNotExists(zk_root_path, ""); + } + }); + + std::shared_ptr metadata_drop_lock; + int32_t drop_lock_version = -1; for (size_t i = 0; i < 1000; ++i) { - std::string stored_metadata_string; - auto exists = client->tryGet(zk_metadata_path, stored_metadata_string); - - if (exists) - { - // this requires same name for columns - // maybe we can do a smarter comparison for columns and primary key expression - if (stored_metadata_string != metadata_string) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", - zk_root_path, - stored_metadata_string); - - auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); - - /// A table on the same Keeper path already exists, we just appended our table id to subscribe as a new replica - /// We still don't know if the table matches the expected metadata so table_is_valid is not changed - /// It will be checked lazily on the first operation - if (code == Coordination::Error::ZOK) - return; - - if (code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); - - /// ZNONODE means we dropped zk_tables_path but didn't finish drop completely - } - - if (client->exists(zk_dropped_path)) - { - LOG_INFO(log, "Removing leftover nodes"); - auto code = client->tryCreate(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral); - - if (code == Coordination::Error::ZNONODE) + bool success = false; + zk_retry.retryLoop( + [&] { - LOG_INFO(log, "Someone else removed leftover nodes"); - } - else if (code == Coordination::Error::ZNODEEXISTS) - { - LOG_INFO(log, "Someone else is removing leftover nodes"); - continue; - } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception::fromPath(code, zk_dropped_lock_path); - } - else - { - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); - if (!dropTable(client, metadata_drop_lock)) - continue; - } - } + auto client = getClient(); + std::string stored_metadata_string; + auto exists = client->tryGet(zk_metadata_path, stored_metadata_string); - Coordination::Requests create_requests - { - zkutil::makeCreateRequest(zk_metadata_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_data_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_tables_path, "", zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_table_path, "", zkutil::CreateMode::Persistent), - }; + if (exists) + { + // this requires same name for columns + // maybe we can do a smarter comparison for columns and primary key expression + if (stored_metadata_string != metadata_string) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", + zk_root_path, + stored_metadata_string); - Coordination::Responses create_responses; - auto code = client->tryMulti(create_requests, create_responses); - if (code == Coordination::Error::ZNODEEXISTS) - { - LOG_INFO(log, "It looks like a table on path {} was created by another server at the same moment, will retry", zk_root_path); - continue; - } - else if (code != Coordination::Error::ZOK) - { - zkutil::KeeperMultiException::check(code, create_requests, create_responses); - } + auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); + /// A table on the same Keeper path already exists, we just appended our table id to subscribe as a new replica + /// We still don't know if the table matches the expected metadata so table_is_valid is not changed + /// It will be checked lazily on the first operation + if (code == Coordination::Error::ZOK) + { + success = true; + return; + } - table_status = TableStatus::VALID; - /// we are the first table created for the specified Keeper path, i.e. we are the first replica - return; + /// We most likely created the path but got a timeout or disconnect + if (code == Coordination::Error::ZNODEEXISTS && zk_retry.isRetry()) + { + success = true; + return; + } + + if (code != Coordination::Error::ZNONODE) + throw zkutil::KeeperException( + code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); + + /// ZNONODE means we dropped zk_tables_path but didn't finish drop completely + } + + if (client->exists(zk_dropped_path)) + { + LOG_INFO(log, "Removing leftover nodes"); + + bool drop_finished = false; + if (zk_retry.isRetry() && metadata_drop_lock != nullptr && drop_lock_version != -1) + { + /// if we have leftover lock from previous try, we need to recreate the ephemeral with our session + Coordination::Requests drop_lock_requests{ + zkutil::makeRemoveRequest(zk_dropped_lock_path, drop_lock_version), + zkutil::makeCreateRequest(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral), + }; + + Coordination::Responses drop_lock_responses; + auto lock_code = client->tryMulti(drop_lock_requests, drop_lock_responses); + if (lock_code == Coordination::Error::ZBADVERSION) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock.reset(); + return; + } + + if (drop_lock_responses[0]->error == Coordination::Error::ZNONODE) + { + /// someone else removed metadata nodes or the previous ephemeral node expired + /// we will try creating dropped lock again to make sure + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock.reset(); + } + else if (lock_code == Coordination::Error::ZOK) + { + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); + drop_lock_version = -1; + Coordination::Stat lock_stat; + client->get(zk_dropped_lock_path, &lock_stat); + drop_lock_version = lock_stat.version; + if (!dropTable(client, metadata_drop_lock)) + { + metadata_drop_lock.reset(); + return; + } + drop_finished = true; + } + } + + if (!drop_finished) + { + auto code = client->tryCreate(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE) + { + LOG_INFO(log, "Someone else removed leftover nodes"); + } + else if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + return; + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, zk_dropped_lock_path); + } + else + { + metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); + drop_lock_version = -1; + Coordination::Stat lock_stat; + client->get(zk_dropped_lock_path, &lock_stat); + drop_lock_version = lock_stat.version; + if (!dropTable(client, metadata_drop_lock)) + { + metadata_drop_lock.reset(); + return; + } + } + } + } + + Coordination::Requests create_requests{ + zkutil::makeCreateRequest(zk_metadata_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_data_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_tables_path, "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_table_path, "", zkutil::CreateMode::Persistent), + }; + + Coordination::Responses create_responses; + auto code = client->tryMulti(create_requests, create_responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO( + log, "It looks like a table on path {} was created by another server at the same moment, will retry", zk_root_path); + return; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, create_requests, create_responses); + } + + table_status = TableStatus::VALID; + /// we are the first table created for the specified Keeper path, i.e. we are the first replica + success = true; + }); + + if (success) + return; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot create metadata for table, because it is removed concurrently or because " - "of wrong zk_root_path ({})", zk_root_path); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot create metadata for table, because it is removed concurrently or because " + "of wrong zk_root_path ({})", + zk_root_path); } @@ -518,7 +606,7 @@ Pipe StorageKeeperMap::read( size_t max_block_size, size_t num_streams) { - checkTable(); + checkTable(context_); storage_snapshot->check(column_names); FieldVectorPtr filtered_keys; @@ -592,13 +680,13 @@ Pipe StorageKeeperMap::read( SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { - checkTable(); + checkTable(local_context); return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); } void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { - checkTable(); + checkTable(local_context); const auto & settings = local_context->getSettingsRef(); ZooKeeperRetriesControl zk_retry{ getName(), @@ -657,7 +745,7 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E void StorageKeeperMap::drop() { - auto current_table_status = getTableStatus(); + auto current_table_status = getTableStatus(getContext()); if (current_table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " @@ -666,7 +754,7 @@ void StorageKeeperMap::drop() } /// if only column metadata is wrong we can still drop the table correctly - if (current_table_status == TableStatus::INVALID_KEEPER_STRUCTURE) + if (current_table_status == TableStatus::INVALID_METADATA) return; auto client = getClient(); @@ -1029,7 +1117,7 @@ UInt64 StorageKeeperMap::keysLimit() const return keys_limit; } -StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const +StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus(const ContextPtr & local_context) const { std::lock_guard lock{init_mutex}; if (table_status != TableStatus::UNKNOWN) @@ -1039,57 +1127,70 @@ StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const { try { - auto client = getClient(); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; - Coordination::Stat metadata_stat; - auto stored_metadata_string = client->get(zk_metadata_path, &metadata_stat); - - if (metadata_stat.numChildren == 0) + zk_retry.retryLoop([&] { + auto client = getClient(); + + Coordination::Stat metadata_stat; + auto stored_metadata_string = client->get(zk_metadata_path, &metadata_stat); + + if (metadata_stat.numChildren == 0) + { + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; + return; + } + + if (metadata_string != stored_metadata_string) + { + LOG_ERROR( + log, + "Table definition does not match to the one stored in the path {}. Stored definition: {}", + zk_root_path, + stored_metadata_string); + table_status = TableStatus::INVALID_METADATA; + return; + } + + // validate all metadata and data nodes are present + Coordination::Requests requests; + requests.push_back(zkutil::makeCheckRequest(zk_table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_data_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_dropped_path, -1)); + + Coordination::Responses responses; + client->tryMulti(requests, responses); + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; - return; - } + if (responses[0]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); + return; + } - if (metadata_string != stored_metadata_string) - { - LOG_ERROR( - log, - "Table definition does not match to the one stored in the path {}. Stored definition: {}", - zk_root_path, - stored_metadata_string); - table_status = TableStatus::INVALID_METADATA; - return; - } + if (responses[1]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Data node ({}) is missing", zk_data_path); + return; + } - // validate all metadata and data nodes are present - Coordination::Requests requests; - requests.push_back(zkutil::makeCheckRequest(zk_table_path, -1)); - requests.push_back(zkutil::makeCheckRequest(zk_data_path, -1)); - requests.push_back(zkutil::makeCheckRequest(zk_dropped_path, -1)); + if (responses[2]->error == Coordination::Error::ZOK) + { + LOG_ERROR(log, "Tables with root node {} are being dropped", zk_root_path); + return; + } - Coordination::Responses responses; - client->tryMulti(requests, responses); - - table_status = TableStatus::INVALID_KEEPER_STRUCTURE; - if (responses[0]->error != Coordination::Error::ZOK) - { - LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); - return; - } - - if (responses[1]->error != Coordination::Error::ZOK) - { - LOG_ERROR(log, "Data node ({}) is missing", zk_data_path); - return; - } - - if (responses[2]->error == Coordination::Error::ZOK) - { - LOG_ERROR(log, "Tables with root node {} are being dropped", zk_root_path); - return; - } - - table_status = TableStatus::VALID; + table_status = TableStatus::VALID; + }); } catch (const Coordination::Exception & e) { @@ -1227,7 +1328,7 @@ void StorageKeeperMap::checkMutationIsPossible(const MutationCommands & commands void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr local_context) { - checkTable(); + checkTable(local_context); if (commands.empty()) return; diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 8ed348a4f6f..1464eeaabad 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -78,9 +78,9 @@ public: UInt64 keysLimit() const; template - void checkTable() const + void checkTable(const ContextPtr & local_context) const { - auto current_table_status = getTableStatus(); + auto current_table_status = getTableStatus(local_context); if (table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " @@ -119,7 +119,7 @@ private: VALID }; - TableStatus getTableStatus() const; + TableStatus getTableStatus(const ContextPtr & context) const; void restoreDataImpl( const BackupPtr & backup, diff --git a/tests/integration/test_keeper_map/configs/keeper_retries.xml b/tests/integration/test_keeper_map/configs/keeper_retries.xml new file mode 100644 index 00000000000..43e5b9a09e8 --- /dev/null +++ b/tests/integration/test_keeper_map/configs/keeper_retries.xml @@ -0,0 +1,14 @@ + + + + 0 + 0 + + + + + + default + + + diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 4b1bcd11cfe..861a7c47687 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -10,6 +10,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/enable_keeper_map.xml"], + user_configs=["configs/keeper_retries.xml"], with_zookeeper=True, stay_alive=True, ) @@ -49,7 +50,6 @@ def assert_keeper_exception_after_partition(query): error = node.query_and_get_error_with_retry( query, sleep_time=1, - settings={"insert_keeper_max_retries": 1, "keeper_max_retries": 1}, ) assert "Coordination::Exception" in error except: @@ -91,7 +91,6 @@ def test_keeper_map_without_zk(started_cluster): error = node.query_and_get_error_with_retry( "SELECT * FROM test_keeper_map_without_zk", sleep_time=1, - settings={"keeper_max_retries": 1}, ) assert "Failed to activate table because of connection issues" in error except: diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml index 0933b6b3031..8406b7db785 100644 --- a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -1,5 +1,6 @@ + 1 0.005 0.005 diff --git a/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml b/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml new file mode 100644 index 00000000000..208dd6e47fa --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml @@ -0,0 +1,14 @@ + + + + 20 + 10000 + + + + + + default + + + diff --git a/tests/integration/test_keeper_map_retries/test.py b/tests/integration/test_keeper_map_retries/test.py index 352119147cd..c6760e5d1a2 100644 --- a/tests/integration/test_keeper_map_retries/test.py +++ b/tests/integration/test_keeper_map_retries/test.py @@ -11,6 +11,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/enable_keeper_map.xml"], + user_configs=["configs/keeper_retries.xml"], with_zookeeper=True, stay_alive=True, ) @@ -42,10 +43,6 @@ def repeat_query(query, repeat): for _ in range(repeat): node.query( query, - settings={ - "keeper_max_retries": 20, - "keeper_retry_max_backoff_ms": 10000, - }, ) @@ -53,10 +50,6 @@ def test_queries(started_cluster): start_clean_clickhouse() node.query("DROP TABLE IF EXISTS keeper_map_retries SYNC") - node.query( - "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a" - ) - node.stop_clickhouse() node.copy_file_to_container( os.path.join(CONFIG_DIR, "fault_injection.xml"), @@ -66,6 +59,10 @@ def test_queries(started_cluster): repeat_count = 10 + node.query( + "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a", + ) + repeat_query( "INSERT INTO keeper_map_retries SELECT number, number FROM numbers(500)", repeat_count, From bababe2dd2fe490e877f9c4ac03d979bd739376a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 15:07:07 +0000 Subject: [PATCH 311/496] Fix remainders I --- src/Functions/FunctionsMultiStringSearch.h | 8 +++--- src/Functions/MultiMatchAnyImpl.h | 27 ++++++++++--------- src/Functions/MultiSearchFirstIndexImpl.h | 16 +++++------ src/Functions/MultiSearchFirstPositionImpl.h | 18 ++++++------- src/Functions/MultiSearchImpl.h | 18 ++++++------- .../0_stateless/00927_disable_hyperscan.sql | 2 +- 6 files changed, 45 insertions(+), 44 deletions(-) diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 03db2651fd0..6bcc8581a38 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -81,7 +81,7 @@ public: return Impl::getReturnType(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr & haystack_ptr = arguments[0].column; const ColumnPtr & needles_ptr = arguments[1].column; @@ -110,13 +110,15 @@ public: col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_const->getValue(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 20b2150048b..06d9eded9c9 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -66,9 +66,10 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorConstant( @@ -81,7 +82,8 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -101,7 +103,7 @@ struct MultiMatchAnyImpl throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); } - res.resize(haystack_offsets.size()); + res.resize(input_rows_count); if (needles_arr.empty()) { @@ -133,9 +135,8 @@ struct MultiMatchAnyImpl /// Once we hit the callback, there is no need to search for others. return 1; }; - const size_t haystack_offsets_size = haystack_offsets.size(); UInt64 offset = 0; - for (size_t i = 0; i < haystack_offsets_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; /// vectorscan restriction. @@ -186,9 +187,10 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorVector( @@ -202,12 +204,13 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); - res.resize(haystack_offsets.size()); + res.resize(input_rows_count); #if USE_VECTORSCAN size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -216,7 +219,7 @@ struct MultiMatchAnyImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); @@ -306,7 +309,7 @@ struct MultiMatchAnyImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 36a5fd514d9..5b34dbfe601 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -33,7 +33,8 @@ struct MultiSearchFirstIndexImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -48,14 +49,13 @@ struct MultiSearchFirstIndexImpl auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -80,10 +80,10 @@ struct MultiSearchFirstIndexImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -92,7 +92,7 @@ struct MultiSearchFirstIndexImpl std::vector needles; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index ccdd82a0ee5..06bf7aa94d8 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -33,7 +33,8 @@ struct MultiSearchFirstPositionImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -52,14 +53,13 @@ struct MultiSearchFirstPositionImpl }; auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -89,10 +89,10 @@ struct MultiSearchFirstPositionImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -106,14 +106,12 @@ struct MultiSearchFirstPositionImpl return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); }; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - { needles.emplace_back(needles_data_string.getDataAt(j).toView()); - } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 467cc96a95f..909425f5a93 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -33,7 +33,8 @@ struct MultiSearchImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -48,14 +49,13 @@ struct MultiSearchImpl auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -79,10 +79,10 @@ struct MultiSearchImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -91,14 +91,12 @@ struct MultiSearchImpl std::vector needles; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - { needles.emplace_back(needles_data_string.getDataAt(j).toView()); - } const auto * const haystack = &haystack_data[prev_haystack_offset]; const size_t haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; diff --git a/tests/queries/0_stateless/00927_disable_hyperscan.sql b/tests/queries/0_stateless/00927_disable_hyperscan.sql index c07848a4fcc..24ec7a35adb 100644 --- a/tests/queries/0_stateless/00927_disable_hyperscan.sql +++ b/tests/queries/0_stateless/00927_disable_hyperscan.sql @@ -1,4 +1,4 @@ --- Tags: no-debug +-- Tags: no-fasttest SET allow_hyperscan = 1; From 0f850952fa304bcc63f7008d4bae54c47ea0564d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 09:52:25 +0000 Subject: [PATCH 312/496] Fix remainders II --- .../FunctionsMultiStringFuzzySearch.h | 8 +++--- src/Functions/MultiMatchAllIndicesImpl.h | 27 +++++++++++-------- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index a92a6570279..8346380c35d 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -71,7 +71,7 @@ public: return Impl::getReturnType(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr & haystack_ptr = arguments[0].column; const ColumnPtr & edit_distance_ptr = arguments[1].column; @@ -114,14 +114,16 @@ public: col_needles_const->getValue(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 3e9c8fba215..3aeac808880 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -52,9 +52,10 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorConstant( @@ -67,7 +68,8 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -87,7 +89,7 @@ struct MultiMatchAllIndicesImpl throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); } - offsets.resize(haystack_offsets.size()); + offsets.resize(input_rows_count); if (needles_arr.empty()) { @@ -114,9 +116,8 @@ struct MultiMatchAllIndicesImpl static_cast*>(context)->push_back(id); return 0; }; - const size_t haystack_offsets_size = haystack_offsets.size(); UInt64 offset = 0; - for (size_t i = 0; i < haystack_offsets_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; /// vectorscan restriction. @@ -146,6 +147,7 @@ struct MultiMatchAllIndicesImpl (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; (void)reject_expensive_hyperscan_regexps; + (void)input_rows_count; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } @@ -160,9 +162,10 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorVector( @@ -176,12 +179,13 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); #if USE_VECTORSCAN - offsets.resize(haystack_offsets.size()); + offsets.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -189,7 +193,7 @@ struct MultiMatchAllIndicesImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); @@ -271,6 +275,7 @@ struct MultiMatchAllIndicesImpl (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; (void)reject_expensive_hyperscan_regexps; + (void)input_rows_count; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } From 99760ad7e4b1b7d4ee86dc2f5aa4df98ead9e260 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 17:17:40 +0000 Subject: [PATCH 313/496] Fix remainders, pt. III --- src/Functions/FunctionsStringSearchToString.h | 4 ++-- src/Functions/FunctionsVisitParam.h | 5 +++-- src/Functions/URL/extractURLParameter.cpp | 7 ++++--- src/Functions/extract.cpp | 7 ++++--- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsStringSearchToString.h b/src/Functions/FunctionsStringSearchToString.h index 978a84de472..c889cf062a3 100644 --- a/src/Functions/FunctionsStringSearchToString.h +++ b/src/Functions/FunctionsStringSearchToString.h @@ -60,7 +60,7 @@ public: return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr column = arguments[0].column; const ColumnPtr column_needle = arguments[1].column; @@ -75,7 +75,7 @@ public: ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); - Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); + Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res, input_rows_count); return col_res; } diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index 5e13fbbad5c..bcaaf0a1c20 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -168,11 +168,12 @@ struct ExtractParamToStringImpl { static void vector(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, std::string needle, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { /// Constant 5 is taken from a function that performs a similar task FunctionsStringSearch.h::ExtractImpl res_data.reserve(haystack_data.size() / 5); - res_offsets.resize(haystack_offsets.size()); + res_offsets.resize(input_rows_count); /// We are looking for a parameter simply as a substring of the form "name" needle = "\"" + needle + "\":"; diff --git a/src/Functions/URL/extractURLParameter.cpp b/src/Functions/URL/extractURLParameter.cpp index f75875e0200..590c2779d9c 100644 --- a/src/Functions/URL/extractURLParameter.cpp +++ b/src/Functions/URL/extractURLParameter.cpp @@ -10,10 +10,11 @@ struct ExtractURLParameterImpl static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string pattern, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.reserve(data.size() / 5); - res_offsets.resize(offsets.size()); + res_offsets.resize(input_rows_count); pattern += '='; const char * param_str = pattern.c_str(); @@ -22,7 +23,7 @@ struct ExtractURLParameterImpl ColumnString::Offset prev_offset = 0; ColumnString::Offset res_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { ColumnString::Offset cur_offset = offsets[i]; diff --git a/src/Functions/extract.cpp b/src/Functions/extract.cpp index 6bbdaff0e3f..c78ee9898b7 100644 --- a/src/Functions/extract.cpp +++ b/src/Functions/extract.cpp @@ -16,10 +16,11 @@ struct ExtractImpl const ColumnString::Offsets & offsets, const std::string & pattern, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.reserve(data.size() / 5); - res_offsets.resize(offsets.size()); + res_offsets.resize(input_rows_count); const OptimizedRegularExpression regexp = Regexps::createRegexp(pattern); @@ -29,7 +30,7 @@ struct ExtractImpl size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t cur_offset = offsets[i]; From 0452768983f379d201e1871ffce63452430bd0c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 08:10:01 +0000 Subject: [PATCH 314/496] Fix remainders, pt. IV --- src/Functions/CountSubstringsImpl.h | 17 ++++----- src/Functions/FunctionsStringSearch.h | 17 ++++++--- src/Functions/FunctionsVisitParam.h | 3 +- src/Functions/HasTokenImpl.h | 5 ++- src/Functions/MatchImpl.h | 54 +++++++++++++-------------- src/Functions/MultiMatchAnyImpl.h | 2 +- src/Functions/PositionImpl.h | 22 +++++------ 7 files changed, 62 insertions(+), 58 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 9ff3e4e1f2a..8e91bc3aeb4 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -37,7 +37,8 @@ struct CountSubstringsImpl const std::string & needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t /*input_rows_count*/) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -150,7 +151,8 @@ struct CountSubstringsImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -158,9 +160,7 @@ struct CountSubstringsImpl ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; @@ -207,7 +207,8 @@ struct CountSubstringsImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -215,9 +216,7 @@ struct CountSubstringsImpl /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { res[i] = 0; auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1); diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index fba6336ebff..7ec0076e395 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -163,7 +163,7 @@ public: return return_type; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const ColumnPtr & column_haystack = (argument_order == ArgumentOrder::HaystackNeedle) ? arguments[0].column : arguments[1].column; const ColumnPtr & column_needle = (argument_order == ArgumentOrder::HaystackNeedle) ? arguments[1].column : arguments[0].column; @@ -236,7 +236,8 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector && col_needle_const) Impl::vectorConstant( col_haystack_vector->getChars(), @@ -244,7 +245,8 @@ public: col_needle_const->getValue(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector_fixed && col_needle_vector) Impl::vectorFixedVector( col_haystack_vector_fixed->getChars(), @@ -253,14 +255,16 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector_fixed && col_needle_const) Impl::vectorFixedConstant( col_haystack_vector_fixed->getChars(), col_haystack_vector_fixed->getN(), col_needle_const->getValue(), vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_const && col_needle_vector) Impl::constantVector( col_haystack_const->getValue(), @@ -268,7 +272,8 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else throw Exception( ErrorCodes::ILLEGAL_COLUMN, diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index bcaaf0a1c20..a77fa740f9c 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -93,7 +93,8 @@ struct ExtractParamImpl std::string needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t /*input_rows_count*/) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index a4ff49859cc..4943bf708c5 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -35,12 +35,13 @@ struct HasTokenImpl const std::string & pattern, const ColumnPtr & start_pos, PaddedPODArray & res, - ColumnUInt8 * res_null) + ColumnUInt8 * res_null, + size_t input_rows_count) { if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; const UInt8 * const begin = haystack_data.data(); diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 55b2fee5400..ceac753fe79 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -127,17 +127,17 @@ struct MatchImpl const String & needle, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack_offsets.size(); - - assert(haystack_size == res.size()); + assert(res.size() == haystack_offsets.size()); + assert(res.size() == input_rows_count); assert(start_pos_ == nullptr); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; /// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack: @@ -202,11 +202,11 @@ struct MatchImpl if (required_substring.empty()) { if (!regexp.getRE2()) /// An empty regexp. Always matches. - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + memset(res.data(), !negate, input_rows_count * sizeof(res[0])); else { size_t prev_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, @@ -291,16 +291,16 @@ struct MatchImpl size_t N, const String & needle, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack.size() / N; + assert(res.size() == haystack.size() / N); + assert(res.size() == input_rows_count); - assert(haystack_size == res.size()); - - if (haystack.empty()) + if (input_rows_count == 0) return; /// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack: @@ -370,11 +370,11 @@ struct MatchImpl if (required_substring.empty()) { if (!regexp.getRE2()) /// An empty regexp. Always matches. - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + memset(res.data(), !negate, input_rows_count * sizeof(res[0])); else { size_t offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack[offset]), N}, @@ -464,18 +464,18 @@ struct MatchImpl const ColumnString::Offsets & needle_offset, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack_offsets.size(); - - assert(haystack_size == needle_offset.size()); - assert(haystack_size == res.size()); + assert(haystack_offsets.size() == needle_offset.size()); + assert(res.size() == haystack_offsets.size()); + assert(res.size() == input_rows_count); assert(start_pos_ == nullptr); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; String required_substr; @@ -488,7 +488,7 @@ struct MatchImpl Regexps::LocalCacheTable cache; Regexps::RegexpPtr regexp; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; @@ -573,15 +573,15 @@ struct MatchImpl const ColumnString::Offsets & needle_offset, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack.size()/N; - - assert(haystack_size == needle_offset.size()); - assert(haystack_size == res.size()); + assert(res.size() == input_rows_count); + assert(res.size() == haystack.size() / N); + assert(res.size() == needle_offset.size()); assert(start_pos_ == nullptr); if (haystack.empty()) @@ -597,7 +597,7 @@ struct MatchImpl Regexps::LocalCacheTable cache; Regexps::RegexpPtr regexp; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack[prev_haystack_offset]; const size_t cur_haystack_length = N; diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 06d9eded9c9..fda752cbacc 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -165,7 +165,7 @@ struct MultiMatchAnyImpl memset(accum.data(), 0, accum.size()); for (size_t j = 0; j < needles.size(); ++j) { - MatchImpl::vectorConstant(haystack_data, haystack_offsets, String(needles[j].data(), needles[j].size()), nullptr, accum, nullptr); + MatchImpl::vectorConstant(haystack_data, haystack_offsets, String(needles[j].data(), needles[j].size()), nullptr, accum, nullptr, input_rows_count); for (size_t i = 0; i < res.size(); ++i) { if constexpr (FindAny) diff --git a/src/Functions/PositionImpl.h b/src/Functions/PositionImpl.h index eeb9d8b6a59..e525b5fab57 100644 --- a/src/Functions/PositionImpl.h +++ b/src/Functions/PositionImpl.h @@ -193,7 +193,8 @@ struct PositionImpl const std::string & needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -214,13 +215,12 @@ struct PositionImpl } ColumnString::Offset prev_offset = 0; - size_t rows = haystack_offsets.size(); if (const ColumnConst * start_pos_const = typeid_cast(&*start_pos)) { /// Needle is empty and start_pos is constant UInt64 start = std::max(start_pos_const->getUInt(0), static_cast(1)); - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t haystack_size = Impl::countChars( reinterpret_cast(pos), reinterpret_cast(pos + haystack_offsets[i] - prev_offset - 1)); @@ -234,7 +234,7 @@ struct PositionImpl else { /// Needle is empty and start_pos is not constant - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t haystack_size = Impl::countChars( reinterpret_cast(pos), reinterpret_cast(pos + haystack_offsets[i] - prev_offset - 1)); @@ -359,7 +359,8 @@ struct PositionImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -367,9 +368,7 @@ struct PositionImpl ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; @@ -423,7 +422,8 @@ struct PositionImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -431,9 +431,7 @@ struct PositionImpl /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; From 59e51de6f8df35a0df0e08e8b3aacd3ba40c4cbd Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 31 Jul 2024 14:58:01 +0000 Subject: [PATCH 315/496] Fix join on tuple with NULLs --- .../Passes/ComparisonTupleEliminationPass.cpp | 10 +++++++++- ...on_tuple_comparison_elimination_bug.reference | 8 ++++++++ ..._join_on_tuple_comparison_elimination_bug.sql | 16 ++++++++++++++++ 3 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference create mode 100644 tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql diff --git a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp index 180470952cd..76dc8ab94b4 100644 --- a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include namespace DB @@ -25,8 +26,15 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - static bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) { + if (parent->getNodeType() == QueryTreeNodeType::JOIN) + { + /// In JOIN ON section comparison of tuples works a bit differently. + /// For example we can join on tuple(NULL) = tuple(NULL), join algorithms consider only NULLs on the top level. + if (parent->as().getJoinExpression().get() == child.get()) + return false; + } return child->getNodeType() != QueryTreeNodeType::TABLE_FUNCTION; } diff --git a/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference new file mode 100644 index 00000000000..c90a64e4a47 --- /dev/null +++ b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference @@ -0,0 +1,8 @@ +1 +\N +1 +\N +1 +\N +1 +\N diff --git a/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql new file mode 100644 index 00000000000..7ef98f88cc7 --- /dev/null +++ b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; + +CREATE TABLE a (key Nullable(String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO a VALUES (NULL), ('1'); + +CREATE TABLE b (key Nullable(String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO b VALUES (NULL), ('1'); + +SELECT a.key FROM a LEFT SEMI JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; +SELECT a.key FROM a LEFT SEMI JOIN b ON a.key IS NOT DISTINCT FROM b.key ORDER BY a.key; +SELECT a.key FROM a LEFT SEMI JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; +SELECT a.key FROM a LEFT ANY JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; + +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; From ca01c1c5691e4562ae6fc71af7b1867cf39f7ad1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 11:33:22 +0200 Subject: [PATCH 316/496] Fix bad merge --- src/Core/SettingsChangesHistory.cpp | 259 ---------------------------- 1 file changed, 259 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c3b9805700f..8f73e10c44f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -497,265 +497,6 @@ static std::initializer_list col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; From 4768e3878552ae0ce9007c1e4f400943a5712825 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 11:37:42 +0200 Subject: [PATCH 317/496] Update ref to 24.7.2 --- ...{02995_baseline_24_7_1.tsv => 02995_baseline_24_7_2.tsv} | 2 +- tests/queries/0_stateless/02995_new_settings_history.sh | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) rename tests/queries/0_stateless/{02995_baseline_24_7_1.tsv => 02995_baseline_24_7_2.tsv} (99%) diff --git a/tests/queries/0_stateless/02995_baseline_24_7_1.tsv b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv similarity index 99% rename from tests/queries/0_stateless/02995_baseline_24_7_1.tsv rename to tests/queries/0_stateless/02995_baseline_24_7_2.tsv index 6c830da8646..10b392f3e04 100644 --- a/tests/queries/0_stateless/02995_baseline_24_7_1.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv @@ -462,7 +462,7 @@ input_format_orc_allow_missing_columns 1 input_format_orc_case_insensitive_column_matching 0 input_format_orc_filter_push_down 1 input_format_orc_import_nested 0 -input_format_orc_read_use_writer_time_zone 0 +input_format_orc_reader_time_zone_name GMT input_format_orc_row_batch_size 100000 input_format_orc_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_orc_use_fast_decoder 1 diff --git a/tests/queries/0_stateless/02995_new_settings_history.sh b/tests/queries/0_stateless/02995_new_settings_history.sh index 917dacc04b0..7fb21f88fae 100755 --- a/tests/queries/0_stateless/02995_new_settings_history.sh +++ b/tests/queries/0_stateless/02995_new_settings_history.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # Note that this is a broad check. A per version check is done in the upgrade test -# Baseline generated with 24.7.1 -# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_1.tsv +# Baseline generated with 24.7.2 +# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_2.tsv $CLICKHOUSE_LOCAL --query " WITH old_settings AS ( - SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_1.tsv', 'TSV', 'name String, default String') + SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_2.tsv', 'TSV', 'name String, default String') ), new_settings AS ( From da3a37c561679daaecbcdece74f92ce98380b2b5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 1 Aug 2024 09:38:27 +0000 Subject: [PATCH 318/496] Improve regex to take into account the xdist name in the instance --- .../test.py | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 9cdf7db2b08..cc0a9022674 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -71,7 +71,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -99,7 +99,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -127,7 +127,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -161,7 +161,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -189,7 +189,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -217,7 +217,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -251,7 +251,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -279,7 +279,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -307,7 +307,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -341,7 +341,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -369,7 +369,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -397,7 +397,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", From c77f6d78d976430faf4353e350d0205bbecf2837 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 12:09:58 +0200 Subject: [PATCH 319/496] Update minio --- tests/integration/compose/docker_compose_minio.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_minio.yml b/tests/integration/compose/docker_compose_minio.yml index 4255a529f6d..40098d05b04 100644 --- a/tests/integration/compose/docker_compose_minio.yml +++ b/tests/integration/compose/docker_compose_minio.yml @@ -2,7 +2,7 @@ version: '2.3' services: minio1: - image: minio/minio:RELEASE.2023-09-30T07-02-29Z + image: minio/minio:RELEASE.2024-07-31T05-46-26Z volumes: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs From 8fc77bec6639c8f0361858d9e031f3cb1175ae30 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:14:14 +0200 Subject: [PATCH 320/496] fix tests --- ...403_enable_extended_results_for_datetime_functions.reference | 2 +- .../0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference index c830d790000..147e368b5c9 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference @@ -64,7 +64,7 @@ toStartOfMonth;toDateTime64;false 2099-07-07 type;toStartOfMonth;toDateTime64;false Date toStartOfWeek;toDate32;false 2099-07-07 type;toStartOfWeek;toDate32;false Date -toStartOfWeek;toDateTime64;false 2099-07-07 +toStartOfWeek;toDateTime64;false 1970-01-01 type;toStartOfWeek;toDateTime64;false Date toMonday;toDate32;false 2099-07-08 type;toMonday;toDate32;false Date diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql index 0f00a52cb86..1769d96aa8d 100644 --- a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql @@ -1,2 +1,2 @@ -SELECT toStartOfWeek(toDateTime64('1970-02-01', 6)); +SELECT toStartOfWeek(toDateTime64('1970-01-01', 6)); SELECT toStartOfWeek(toDateTime('1970-01-01')); From d0b514f12843fd09f18888666aa98170a52fb9d5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Aug 2024 13:27:09 +0200 Subject: [PATCH 321/496] Review fixes --- src/Interpreters/Cache/FileCache.cpp | 12 ++++++------ src/Interpreters/Cache/IFileCachePriority.h | 6 +++--- src/Interpreters/Cache/LRUFileCachePriority.cpp | 10 +++++----- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 6 +++--- src/Interpreters/Cache/SLRUFileCachePriority.h | 2 +- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index bf8dd24a1db..1f79c7d9032 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -998,7 +998,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() FileCacheReserveStat stat; EvictionCandidates eviction_candidates; - IFileCachePriority::DesiredSizeStatus desired_size_status; + IFileCachePriority::CollectStatus desired_size_status; try { /// Collect at most `keep_up_free_space_remove_batch` elements to evict, @@ -1009,7 +1009,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() #ifdef DEBUG_OR_SANITIZER_BUILD /// Let's make sure that we correctly processed the limits. - if (desired_size_status == IFileCachePriority::DesiredSizeStatus::SUCCESS + if (desired_size_status == IFileCachePriority::CollectStatus::SUCCESS && eviction_candidates.size() < keep_up_free_space_remove_batch) { const auto current_size = main_priority->getSize(lock); @@ -1070,13 +1070,13 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() [[maybe_unused]] bool scheduled = false; switch (desired_size_status) { - case IFileCachePriority::DesiredSizeStatus::SUCCESS: [[fallthrough]]; - case IFileCachePriority::DesiredSizeStatus::CANNOT_EVICT: + case IFileCachePriority::CollectStatus::SUCCESS: [[fallthrough]]; + case IFileCachePriority::CollectStatus::CANNOT_EVICT: { scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); break; } - case IFileCachePriority::DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT: + case IFileCachePriority::CollectStatus::REACHED_MAX_CANDIDATES_LIMIT: { scheduled = keep_up_free_space_ratio_task->schedule(); break; @@ -1558,7 +1558,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheReserveStat stat; if (main_priority->collectCandidatesForEviction( new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, - stat, eviction_candidates, cache_lock) == IFileCachePriority::DesiredSizeStatus::SUCCESS) + stat, eviction_candidates, cache_lock) == IFileCachePriority::CollectStatus::SUCCESS) { if (eviction_candidates.size() == 0) { diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 9885ab00f78..6970d02473a 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -150,14 +150,14 @@ public: /// Collect eviction candidates sufficient to have `desired_size` /// and `desired_elements_num` as current cache state. /// Collect no more than `max_candidates_to_evict` elements. - /// Return `true` if the first condition is satisfied. - enum class DesiredSizeStatus + /// Return SUCCESS status if the first condition is satisfied. + enum class CollectStatus { SUCCESS, CANNOT_EVICT, REACHED_MAX_CANDIDATES_LIMIT, }; - virtual DesiredSizeStatus collectCandidatesForEviction( + virtual CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 7970eaa3e13..0e0170c76e3 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -323,7 +323,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::CollectStatus LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -336,23 +336,23 @@ IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesFor return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock, &desired_size, &desired_elements_count); }; - auto status = DesiredSizeStatus::CANNOT_EVICT; + auto status = CollectStatus::CANNOT_EVICT; auto stop_condition = [&]() { if (desired_limits_satisfied()) { - status = DesiredSizeStatus::SUCCESS; + status = CollectStatus::SUCCESS; return true; } if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) { - status = DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT; + status = CollectStatus::REACHED_MAX_CANDIDATES_LIMIT; return true; } return false; }; iterateForEviction(res, stat, stop_condition, lock); - chassert(status != DesiredSizeStatus::SUCCESS || stop_condition()); + chassert(status != CollectStatus::SUCCESS || stop_condition()); return status; } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 9bced106727..0ca62b19d37 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -63,7 +63,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - DesiredSizeStatus collectCandidatesForEviction( + CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index dc0df223cb0..f5ea519d7d4 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -256,7 +256,7 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::CollectStatus SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -285,7 +285,7 @@ IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesFo chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (probationary_desired_size_status == DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT) + if (probationary_desired_size_status == CollectStatus::REACHED_MAX_CANDIDATES_LIMIT) return probationary_desired_size_status; const auto desired_protected_size = getRatio(desired_size, size_ratio); @@ -306,7 +306,7 @@ IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesFo desired_protected_size, desired_protected_elements_num, protected_queue.getStateInfoForLog(lock)); - if (probationary_desired_size_status == DesiredSizeStatus::SUCCESS) + if (probationary_desired_size_status == CollectStatus::SUCCESS) return protected_desired_size_status; else return probationary_desired_size_status; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index e6d20e0d0ee..23bc8c0908b 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,7 +58,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - DesiredSizeStatus collectCandidatesForEviction( + CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, From 2c018d6f4d4d6fa1c04e91306b60b6e85d8e468f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:38:41 +0200 Subject: [PATCH 322/496] Update cluster.py --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 90b28a4cda3..acf033de46d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -248,7 +248,7 @@ def check_rabbitmq_is_available(rabbitmq_id, cookie): ), stdout=subprocess.PIPE, ) - p.communicate() + p.wait(timeout=60) return p.returncode == 0 From 53d93b177e371b8dcc13e404f48209e3df456fe4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Aug 2024 11:56:07 +0000 Subject: [PATCH 323/496] Fix 02434_cancel_insert_when_client_dies --- .../02434_cancel_insert_when_client_dies.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index f85aaed7716..45f4194104e 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -10,22 +10,26 @@ export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE -$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' +$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A + settings non_replicated_deduplication_window=1000 + , merge_tree_clear_old_temporary_directories_interval_seconds = 1 + ;" $CLICKHOUSE_CLIENT -q "create table dedup_dist(A Int64) Engine = Distributed('test_cluster_one_shard_two_replicas', currentDatabase(), dedup_test)" function insert_data { - SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000" + # send_logs_level: https://github.com/ClickHouse/ClickHouse/issues/67599 + SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000&send_logs_level=fatal" # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" TYPE=$(( RANDOM % 5 )) if [[ "$TYPE" -eq 0 ]]; then # client will send 10000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) - $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" \ + $CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=fatal --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" \ -q 'insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 1 ]]; then - $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" --prefer_localhost_replica="$(( RANDOM % 2))" \ + $CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=fatal --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" --prefer_localhost_replica="$(( RANDOM % 2))" \ -q 'insert into dedup_dist settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 2 ]]; then $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE From 029deaeee8431d0ef6f2a460c2bd8631c8025254 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 14:13:11 +0200 Subject: [PATCH 324/496] Fix 02910_bad_logs_level_in_local in fast tests! --- tests/queries/0_stateless/02910_bad_logs_level_in_local.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh index badf7232a95..b5de10bf191 100755 --- a/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh +++ b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh @@ -1,14 +1,14 @@ #!/usr/bin/expect -f log_user 0 -set timeout 60 +set timeout 30 match_max 100000 spawn bash -c "clickhouse-local" expect ":) " send -- "SET send_logs_level = 't'\r" -expect "Exception on client:" +expect "Unexpected value of LogsLevel:" {} timeout {exit 1} expect ":) " send -- "exit\r" expect eof From bcc75d3681d45b6637211aca0367703b3e957c05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 14:38:41 +0200 Subject: [PATCH 325/496] Make 02477_analyzer_function_hints.sh parallelizable --- .../0_stateless/02477_analyzer_function_hints.sh | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.sh b/tests/queries/0_stateless/02477_analyzer_function_hints.sh index d49c20cab75..f83935e47fb 100755 --- a/tests/queries/0_stateless/02477_analyzer_function_hints.sh +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel - set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -14,11 +12,11 @@ $CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS allow_experimental_analyzer = 1 $CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['uniqExact'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function;" -$CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function AS x -> x + 1;" -$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ - | grep "Maybe you meant: \['test_user_defined_function'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function"; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function_$CLICKHOUSE_DATABASE;" +$CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE AS x -> x + 1;" +$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_function_${CLICKHOUSE_DATABASE}A(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep -E "Maybe you meant: \[.*'test_user_defined_function_$CLICKHOUSE_DATABASE'.*\]" &>/dev/null; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE"; $CLICKHOUSE_CLIENT -q "WITH (x -> x + 1) AS lambda_function SELECT lambda_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['lambda_function'\]" &>/dev/null; From 008408c81f23ed615cb899048418dc46aa3c2a9f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 13:47:34 +0100 Subject: [PATCH 326/496] impl --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 9b96ce3e586..aaeee466794 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage +-- Tags: no-random-merge-tree-settings, no-random-settings, no-tsan, no-debug, no-object-storage, long -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher @@ -16,7 +16,7 @@ CREATE TABLE adaptive_table( value String ) ENGINE MergeTree() ORDER BY key -SETTINGS index_granularity_bytes=1048576, +SETTINGS index_granularity_bytes = 1048576, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, enable_vertical_merge_algorithm = 0; From 70228acd7e809230582883a0b6b70c4cd9c04daa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Aug 2024 15:02:29 +0200 Subject: [PATCH 327/496] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a4c873ba3f9..5cd4200d9ad 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,7 +22,6 @@ #### New Feature * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). -* Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). * Support JWT authentication in `clickhouse-client` (will be available only in ClickHouse Cloud). [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). * Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). From 48c6e36dfd23f297907575ce4696f761aec49e11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:03:41 +0200 Subject: [PATCH 328/496] Make 01062_window_view_event_hop_watch_asc parallelizable --- .../01062_window_view_event_hop_watch_asc.py | 28 ++++++++----------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index d6cc3ee1a88..3c85ff30ba8 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -1,5 +1,4 @@ #!/usr/bin/env python3 -# Tags: no-parallel import os import sys @@ -17,6 +16,7 @@ log = None with client(name="client1>", log=log) as client1, client( name="client2>", log=log ) as client2: + database_name = os.environ["CLICKHOUSE_DATABASE"] client1.expect(prompt) client2.expect(prompt) @@ -31,40 +31,38 @@ with client(name="client1>", log=log) as client1, client( client2.send("SET allow_experimental_analyzer = 0") client2.expect(prompt) - client1.send("CREATE DATABASE IF NOT EXISTS 01062_window_view_event_hop_watch_asc") + client1.send(f"DROP TABLE IF EXISTS {database_name}.mt") client1.expect(prompt) - client1.send("DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.mt") - client1.expect(prompt) - client1.send("DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.wv SYNC") + client1.send(f"DROP TABLE IF EXISTS {database_name}.wv SYNC") client1.expect(prompt) client1.send( - "CREATE TABLE 01062_window_view_event_hop_watch_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" + f"CREATE TABLE {database_name}.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" ) client1.expect(prompt) client1.send( - "CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid" + f"CREATE WINDOW VIEW {database_name}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM {database_name}.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid" ) client1.expect(prompt) - client1.send("WATCH 01062_window_view_event_hop_watch_asc.wv") + client1.send(f"WATCH {database_name}.wv") client1.expect("Query id" + end_of_block) client1.expect("Progress: 0.00 rows.*\\)") client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) client2.expect(prompt) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));" ) client2.expect(prompt) client1.expect("1*" + end_of_block) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" ) client2.expect(prompt) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));" ) client2.expect(prompt) client1.expect("1" + end_of_block) @@ -77,9 +75,7 @@ with client(name="client1>", log=log) as client1, client( if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) - client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.wv SYNC") + client1.send(f"DROP TABLE {database_name}.wv SYNC") client1.expect(prompt) - client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.mt") - client1.expect(prompt) - client1.send("DROP DATABASE IF EXISTS 01062_window_view_event_hop_watch_asc") + client1.send(f"DROP TABLE {database_name}.mt") client1.expect(prompt) From 7b72362e99a093a4f880d333c1b50cd114b590c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:06:27 +0200 Subject: [PATCH 329/496] 01493_alter_remove_properties_zookeeper is already parallelizable --- .../0_stateless/01493_alter_remove_properties_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql index 92e6fce2c93..362da3ac364 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql @@ -1,4 +1,4 @@ --- Tags: zookeeper, no-parallel +-- Tags: zookeeper DROP TABLE IF EXISTS r_prop_table1; DROP TABLE IF EXISTS r_prop_table2; From 56e48cf43b4d13810e2bb4b4e941954b654a1cb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:09:56 +0200 Subject: [PATCH 330/496] Make 01676_range_hashed_dictionary parallelizable --- .../01676_range_hashed_dictionary.sql | 92 +++++++++---------- 1 file changed, 42 insertions(+), 50 deletions(-) diff --git a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql index 430f3a86dc1..ba2a9eba87f 100644 --- a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql +++ b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql @@ -1,10 +1,4 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS database_for_range_dict; - -CREATE DATABASE database_for_range_dict; - -CREATE TABLE database_for_range_dict.date_table +CREATE TABLE date_table ( CountryID UInt64, StartDate Date, @@ -14,11 +8,11 @@ CREATE TABLE database_for_range_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_range_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); +INSERT INTO date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); -CREATE DICTIONARY database_for_range_dict.range_dictionary +CREATE DICTIONARY range_dictionary ( CountryID UInt64, StartDate Date, @@ -26,7 +20,7 @@ CREATE DICTIONARY database_for_range_dict.range_dictionary Tax Float64 DEFAULT 0.2 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_range_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate) @@ -35,30 +29,30 @@ SETTINGS(dictionary_use_async_executor=1, max_threads=8) SELECT 'Dictionary not nullable'; SELECT 'dictGet'; -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31')); -SELECT dictGetOrDefault('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetOrDefault('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); SELECT 'dictHas'; -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(1), toDate('2019-05-15')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(1), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(2), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(2), toDate('2019-05-31')); +SELECT dictHas('range_dictionary', toUInt64(1), toDate('2019-05-15')); +SELECT dictHas('range_dictionary', toUInt64(1), toDate('2019-05-29')); +SELECT dictHas('range_dictionary', toUInt64(2), toDate('2019-05-29')); +SELECT dictHas('range_dictionary', toUInt64(2), toDate('2019-05-31')); SELECT 'select columns from dictionary'; SELECT 'allColumns'; -SELECT * FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT * FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'noColumns'; -SELECT 1 FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT 1 FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumns'; -SELECT CountryID, StartDate, Tax FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT CountryID, StartDate, Tax FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumn'; -SELECT Tax FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT Tax FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; -DROP DICTIONARY database_for_range_dict.range_dictionary; -DROP TABLE database_for_range_dict.date_table; +DROP DICTIONARY range_dictionary; +DROP TABLE date_table; -CREATE TABLE database_for_range_dict.date_table +CREATE TABLE date_table ( CountryID UInt64, StartDate Date, @@ -68,11 +62,11 @@ CREATE TABLE database_for_range_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_range_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), NULL); +INSERT INTO date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), NULL); -CREATE DICTIONARY database_for_range_dict.range_dictionary_nullable +CREATE DICTIONARY range_dictionary_nullable ( CountryID UInt64, StartDate Date, @@ -80,35 +74,33 @@ CREATE DICTIONARY database_for_range_dict.range_dictionary_nullable Tax Nullable(Float64) DEFAULT 0.2 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_range_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); SELECT 'Dictionary nullable'; SELECT 'dictGet'; -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31')); -SELECT dictGetOrDefault('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetOrDefault('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); SELECT 'dictHas'; -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(1), toDate('2019-05-15')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(1), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(2), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(2), toDate('2019-05-31')); +SELECT dictHas('range_dictionary_nullable', toUInt64(1), toDate('2019-05-15')); +SELECT dictHas('range_dictionary_nullable', toUInt64(1), toDate('2019-05-29')); +SELECT dictHas('range_dictionary_nullable', toUInt64(2), toDate('2019-05-29')); +SELECT dictHas('range_dictionary_nullable', toUInt64(2), toDate('2019-05-31')); SELECT 'select columns from dictionary'; SELECT 'allColumns'; -SELECT * FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT * FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'noColumns'; -SELECT 1 FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT 1 FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumns'; -SELECT CountryID, StartDate, Tax FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT CountryID, StartDate, Tax FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumn'; -SELECT Tax FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT Tax FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; -DROP DICTIONARY database_for_range_dict.range_dictionary_nullable; -DROP TABLE database_for_range_dict.date_table; - -DROP DATABASE database_for_range_dict; +DROP DICTIONARY range_dictionary_nullable; +DROP TABLE date_table; From bb7039eeec01ce59008103727f8c03ddd26a3d29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:17:00 +0200 Subject: [PATCH 331/496] Make 01107_atomic_db_detach_attach parallelizable --- .../01107_atomic_db_detach_attach.sh | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index bcaa70abbb5..e9879344259 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -1,29 +1,30 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" +NEW_DATABASE=test_01107_${CLICKHOUSE_DATABASE} +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS ${NEW_DATABASE}" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${NEW_DATABASE} ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${NEW_DATABASE}.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${NEW_DATABASE}.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 -$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 -$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" -$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -q "DETACH TABLE ${NEW_DATABASE}.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "ATTACH TABLE ${NEW_DATABASE}.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK" wait -$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 -$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" +$CLICKHOUSE_CLIENT -q "ATTACH TABLE ${NEW_DATABASE}.mt" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${NEW_DATABASE}.mt" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE ${NEW_DATABASE}" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${NEW_DATABASE}.mt" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${NEW_DATABASE}.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait From 1d85f9b1cba3c8fe168286a660d3c0a4fd471a95 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 1 Aug 2024 14:42:58 +0100 Subject: [PATCH 332/496] fix remove_local_directory_contents --- tests/integration/test_storage_delta/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index e485bc90ee0..384b8296f66 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -52,9 +52,13 @@ def get_spark(): return builder.master("local").getOrCreate() -def remove_local_directory_contents(local_path): - for local_file in glob.glob(local_path + "/**"): - os.unlink(local_file) +def remove_local_directory_contents(full_path): + for path in glob.glob(f"{full_path}/**"): + if os.path.isfile(path): + os.unlink(path) + else: + remove_local_directory_contents(path) + os.rmdir(path) @pytest.fixture(scope="module") From 0913f0189ba350236d32d774265770b654860a80 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Wed, 31 Jul 2024 09:06:30 +0000 Subject: [PATCH 333/496] Don't count a search query as a search pattern match --- tests/integration/helpers/cluster.py | 10 +++++++--- .../integration/test_mask_sensitive_info/test.py | 15 +++++++-------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 90b28a4cda3..6bc0ece63ca 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3922,7 +3922,11 @@ class ClickHouseInstance: ) def contains_in_log( - self, substring, from_host=False, filename="clickhouse-server.log" + self, + substring, + from_host=False, + filename="clickhouse-server.log", + exclusion_substring="", ): if from_host: # We check fist file exists but want to look for all rotated logs as well @@ -3930,7 +3934,7 @@ class ClickHouseInstance: [ "bash", "-c", - f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{substring}" {self.logs_dir}/{filename}* || true', + f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{substring}" {self.logs_dir}/{filename}* | ( [ -z "{exclusion_substring}" ] && cat || grep -v "${exclusion_substring}" ) || true', ] ) else: @@ -3938,7 +3942,7 @@ class ClickHouseInstance: [ "bash", "-c", - f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{substring}" /var/log/clickhouse-server/{filename} || true', + f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{substring}" /var/log/clickhouse-server/{filename} | ( [ -z "{exclusion_substring}" ] && cat || grep -v "${exclusion_substring}" ) || true', ] ) return len(result) > 0 diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 902d3800324..6f6dc4d287f 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -13,6 +13,7 @@ node = cluster.add_instance( with_zookeeper=True, with_azurite=True, ) +base_search_query = "SELECT COUNT() FROM system.query_log WHERE query LIKE " @pytest.fixture(scope="module", autouse=True) @@ -35,7 +36,7 @@ def check_logs(must_contain=[], must_not_contain=[]): .replace("]", "\\]") .replace("*", "\\*") ) - assert node.contains_in_log(escaped_str) + assert node.contains_in_log(escaped_str, exclusion_substring=base_search_query) for str in must_not_contain: escaped_str = ( @@ -44,7 +45,9 @@ def check_logs(must_contain=[], must_not_contain=[]): .replace("]", "\\]") .replace("*", "\\*") ) - assert not node.contains_in_log(escaped_str) + assert not node.contains_in_log( + escaped_str, exclusion_substring=base_search_query + ) for str in must_contain: escaped_str = str.replace("'", "\\'") @@ -60,7 +63,7 @@ def system_query_log_contains_search_pattern(search_pattern): return ( int( node.query( - f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{search_pattern}%'" + f"{base_search_query}'%{search_pattern}%' AND query NOT LIKE '{base_search_query}%'" ).strip() ) >= 1 @@ -105,7 +108,6 @@ def test_create_alter_user(): must_not_contain=[ password, "IDENTIFIED BY", - "IDENTIFIED BY", "IDENTIFIED WITH plaintext_password BY", ], ) @@ -366,10 +368,7 @@ def test_table_functions(): f"remoteSecure(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", f"s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", f"s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '{password}')", - ( - f"deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", - "DNS_ERROR", - ), + f"deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", From 048fbacc40062b05510916134c9d9525e7fab63a Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 1 Aug 2024 16:48:19 +0200 Subject: [PATCH 334/496] Update README.md --- README.md | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 98f9108f14c..2120a4d1211 100644 --- a/README.md +++ b/README.md @@ -34,17 +34,13 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.7 Community Call](https://clickhouse.com/company/events/v24-7-community-release-call) - Jul 30 +* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 29 ## Upcoming Events Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9 -* [ClickHouse Cloud - Live Update Call](https://clickhouse.com/company/events/202407-cloud-update-live) - Jul 9 -* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9 -* [AWS Summit in New York](https://clickhouse.com/company/events/2024-07-awssummit-nyc) - Jul 10 -* [ClickHouse Meetup @ Klaviyo - Boston](https://www.meetup.com/clickhouse-boston-user-group/events/300907870) - Jul 11 +* MORE COMING SOON! ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 582bcfdc03e1124500325c3104497719473657cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 16:57:41 +0200 Subject: [PATCH 335/496] Add no-parallel back to 01107_atomic_db_detach_attach --- tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e9879344259..a6a99aadac2 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel +# no-parallel: FIXME: Timing issues with INSERT + DETACH (https://github.com/ClickHouse/ClickHouse/pull/67610/files#r1700345054) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0978441a0261c6003c7a9f4661ac87138e909622 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 336/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 53cbb4811047cad2bdf2b882bc89ff9a83ac4577 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:06:05 +0200 Subject: [PATCH 337/496] Try fix 03143_asof_join_ddb_long --- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..0b17ade5d1c 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-random-merge-tree-settings +-- no-random-merge-tree-settings - times out in private DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From 9362d1a5668bcd6e4e629ab26ec44d4bc8cb6513 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 15:04:13 +0200 Subject: [PATCH 338/496] CI: Create release workflow dry run fix fix --- .github/actions/release/action.yml | 30 +++--- .github/workflows/create_release.yml | 144 +++++++++++++++++++++++++-- tests/ci/create_release.py | 2 +- 3 files changed, 153 insertions(+), 23 deletions(-) diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml index c3897682a33..a287aa8b41d 100644 --- a/.github/actions/release/action.yml +++ b/.github/actions/release/action.yml @@ -16,8 +16,7 @@ inputs: - new dry-run: description: 'Dry run' - required: false - default: true + required: true type: boolean token: required: true @@ -30,8 +29,7 @@ runs: shell: bash run: | python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ - ${{ inputs.dry-run && '--dry-run' || '' }} + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} echo "::group::Release Info" python3 -m json.tool /tmp/release_info.json echo "::endgroup::" @@ -44,20 +42,20 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push Git Tag for the Release shell: bash run: | - python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push New Release Branch if: ${{ inputs.type == 'new' }} shell: bash run: | - python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump CH Version and Update Contributors' List shell: bash run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash @@ -107,37 +105,37 @@ runs: shell: bash if: ${{ inputs.type == 'patch' }} run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export RPM Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test RPM Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export Debian Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test Debian Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Docker clickhouse/clickhouse-server building if: ${{ inputs.type == 'patch' }} shell: bash @@ -165,4 +163,4 @@ runs: if: ${{ !cancelled() }} shell: bash run: | - python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 55644bdd503..217f27086c5 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -35,10 +35,142 @@ jobs: with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} fetch-depth: 0 - - name: Call Release Action - uses: ./.github/actions/release + - name: Prepare Release Info + shell: bash + run: | + python3 ./tests/ci/create_release.py --prepare-release-info \ + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} + echo "::group::Release Info" + python3 -m json.tool /tmp/release_info.json + echo "::endgroup::" + release_tag=$(jq -r '.release_tag' /tmp/release_info.json) + commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + echo "Release Tag: $release_tag" + echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" + echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + - name: Download All Release Artifacts + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Push Git Tag for the Release + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Push New Release Branch + if: ${{ inputs.type == 'new' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump Docker versions, Changelog, Security + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + git checkout master + python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" + echo "List versions" + ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv + echo "Update docker version" + ./utils/list-versions/update-docker-version.sh + echo "Generate ChangeLog" + export CI=1 + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ + --volume=".:/ClickHouse" clickhouse/style-test \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --gh-user-or-token=${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} --jobs=5 \ + --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md + echo "Generate Security" + python3 ./utils/security-generator/generate_security.py > SECURITY.md + git diff HEAD + - name: Create ChangeLog PR + if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + uses: peter-evans/create-pull-request@v6 with: - ref: ${{ inputs.ref }} - type: ${{ inputs.type }} - dry-run: ${{ inputs.dry-run }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + author: "robot-clickhouse " + token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} + committer: "robot-clickhouse " + commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + branch: auto/${{ env.RELEASE_TAG }} + assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher + delete-branch: true + title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} + labels: do not test + body: | + Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + ### Changelog category (leave one): + - Not for changelog (changelog entry is not required) + - name: Complete previous steps and Restore git state + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --set-progress-completed + git reset --hard HEAD + git checkout "$GITHUB_REF_NAME" + - name: Create GH Release + shell: bash + if: ${{ inputs.type == 'patch' }} + run: | + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Docker clickhouse/clickhouse-server building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker server release" + export CHECK_NAME="Docker server image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./create_release.py --set-progress-completed + - name: Docker clickhouse/clickhouse-keeper building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker keeper release" + export CHECK_NAME="Docker keeper image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./create_release.py --set-progress-completed + - name: Set current Release progress to Completed with OK + shell: bash + run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" + python3 ./tests/ci/create_release.py --set-progress-completed + - name: Post Slack Message + if: ${{ !cancelled() }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index a0b4083b673..b02a0bb8ed5 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -315,7 +315,7 @@ class ReleaseInfo: cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" - cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file '{body_file} --label 'do not test' --assignee @{actor}" + cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) Shell.run(cmd_push_branch, check=True, dry_run=dry_run) Shell.run(cmd_create_pr, check=True, dry_run=dry_run) From c534cd5bc21b59788750bdfcfb4177ebba0afc85 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 16:22:43 +0200 Subject: [PATCH 339/496] changelog.py to retrieve best token s3fs fix changelog.py to use base branch to filter prs --- .github/workflows/create_release.yml | 12 ++++++------ tests/ci/artifactory.py | 4 ++++ tests/ci/changelog.py | 23 +++++++++++++++++++++++ 3 files changed, 33 insertions(+), 6 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 217f27086c5..5e34f50fab5 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -81,7 +81,7 @@ jobs: docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ --volume=".:/ClickHouse" clickhouse/style-test \ /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token=${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} --jobs=5 \ + --jobs=5 \ --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md echo "Generate Security" @@ -111,11 +111,11 @@ jobs: python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - shell: bash - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} +# - name: Create GH Release +# shell: bash +# if: ${{ inputs.type == 'patch' }} +# run: | +# python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 86dcaf79854..4ee326593e6 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -52,6 +52,10 @@ class R2MountPoint: if self.CACHE_ENABLED else "" ) + if not dry_run: + self.aux_mount_options += ( + "-o passwd_file /home/ubuntu/.passwd-s3fs_packages " + ) # without -o nomultipart there are errors like "Error 5 writing to /home/ubuntu/***.deb: Input/output error" self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" elif self.app == MountPointApp.RCLONE: diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 3ba618f3ae5..e23dd8e4c67 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -19,6 +19,8 @@ from env_helper import TEMP_PATH from git_helper import git_runner, is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from ci_utils import Shell from version_helper import ( FILE_WITH_VERSION_PATH, get_abs_path, @@ -171,6 +173,7 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--gh-user-or-token", help="user name or GH token to authenticate", + default=get_best_robot_token(), ) parser.add_argument( "--gh-password", @@ -397,6 +400,15 @@ def get_year(prs: PullRequests) -> int: return max(pr.created_at.year for pr in prs) +def get_branch_by_tag(tag: str) -> Optional[str]: + tag.removeprefix("v") + versions = tag.split(".") + if len(versions) < 3: + print("ERROR: Can't get branch by tag") + return None + return f"{versions[0]}.{versions[1]}" + + def main(): log_levels = [logging.WARN, logging.INFO, logging.DEBUG] args = parse_args() @@ -446,6 +458,17 @@ def main(): gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" + branch = get_branch_by_tag(TO_REF) + if branch and Shell.check(f"git show-ref --quiet {branch}"): + try: + if int(branch.split(".")[-1]) > 1: + query += f" base:{branch}" + print(f"NOTE: will use base branch to filter PRs {branch}") + except ValueError: + print(f"ERROR: cannot get minor version from branch {branch} - pass") + pass + else: + print(f"ERROR: invalid branch {branch} - pass") prs = gh.get_pulls_from_search( query=query, merged=merged, sort="created", progress_func=tqdm.tqdm ) From 8214910cc7bd84f613631c2fada9682820df8003 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 20:14:22 +0200 Subject: [PATCH 340/496] add geesfs --- .github/workflows/create_release.yml | 10 +++---- .github/workflows/release.yml | 4 +-- tests/ci/artifactory.py | 35 +++++++++++++++++++++--- tests/ci/changelog.py | 41 ++++++++++++++++++---------- tests/ci/ci_utils.py | 11 ++++---- 5 files changed, 69 insertions(+), 32 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 5e34f50fab5..c3126abe461 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -62,15 +62,14 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - shell: bash - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} +# - name: Bump CH Version and Update Contributors' List +# shell: bash +# run: | +# python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash run: | - git checkout master python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv @@ -96,6 +95,7 @@ jobs: committer: "robot-clickhouse " commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} branch: auto/${{ env.RELEASE_TAG }} + base: master assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher delete-branch: true title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 3bd6dfae6ca..8620d15ec19 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -55,12 +55,12 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --check-name "$CHECK_NAME" --push + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push - name: Check docker clickhouse/clickhouse-keeper building run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --check-name "$CHECK_NAME" --push + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push - name: Cleanup if: always() run: | diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 4ee326593e6..a508374f856 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -15,6 +15,7 @@ from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): RCLONE = "rclone" S3FS = "s3fs" + GEESEFS = "geesefs" class R2MountPoint: @@ -70,6 +71,20 @@ class R2MountPoint: ) # Use --no-modtime to try to avoid: ERROR : rpm/lts/clickhouse-client-24.3.6.5.x86_64.rpm: Failed to apply pending mod time self.mount_cmd = f"rclone mount remote:{self.bucket_name} {self.MOUNT_POINT} --daemon --cache-dir {self.cache_dir} --umask 0000 --log-file {self.LOG_FILE} {self.aux_mount_options}" + elif self.app == MountPointApp.GEESEFS: + self.cache_dir = "/home/ubuntu/geesefs_cache" + self.aux_mount_options += ( + f" --cache={self.cache_dir} " if self.CACHE_ENABLED else "" + ) + if not dry_run: + self.aux_mount_options += f" --shared-config=/home/ubuntu/.r2_auth " + else: + self.aux_mount_options += ( + f" --shared-config=/home/ubuntu/.r2_auth_test " + ) + if self.DEBUG: + self.aux_mount_options += " --debug_s3 --debug_fuse " + self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=2050 --gc-interval=100 --max-flushers=5 --max-parallel-parts=1 --max-parallel-copy=2 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" else: assert False @@ -87,7 +102,7 @@ class R2MountPoint: Shell.run(_UNMOUNT_CMD) Shell.run(_MKDIR_CMD) Shell.run(_MKDIR_FOR_CACHE) - if self.app == MountPointApp.S3FS: + if self.app != MountPointApp.RCLONE: Shell.run(self.mount_cmd, check=True) else: # didn't manage to use simple run() and without blocking or failure @@ -158,7 +173,13 @@ class DebianArtifactory: cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' print("Running test command:") print(f" {cmd}") - Shell.run(cmd, check=True) + assert Shell.check(cmd) + print(f"Test packages installation, version [latest]") + debian_command_2 = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static clickhouse-client" + cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command_2}"' + print("Running test command:") + print(f" {cmd}") + assert Shell.check(cmd) self.release_info.debian_command = debian_command self.release_info.dump() @@ -234,7 +255,13 @@ class RpmArtifactory: cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' print("Running test command:") print(f" {cmd}") - Shell.run(cmd, check=True) + assert Shell.check(cmd) + print(f"Test package installation, version [latest]") + rpm_command_2 = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client" + cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command_2}"' + print("Running test command:") + print(f" {cmd}") + assert Shell.check(cmd) self.release_info.rpm_command = rpm_command self.release_info.dump() @@ -350,7 +377,7 @@ if __name__ == "__main__": ERROR : IO error: NotImplemented: versionId not implemented Failed to copy: NotImplemented: versionId not implemented """ - mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) + mp = R2MountPoint(MountPointApp.GEESEFS, dry_run=args.dry_run) if args.export_debian: with ReleaseContextManager( release_progress=ReleaseProgress.EXPORT_DEB diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index e23dd8e4c67..929f0f3523a 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -7,7 +7,7 @@ import re from datetime import date, timedelta from pathlib import Path from subprocess import DEVNULL -from typing import Any, Dict, List, Optional, TextIO +from typing import Any, Dict, List, Optional, TextIO, Tuple import tqdm # type: ignore from github.GithubException import RateLimitExceededException, UnknownObjectException @@ -400,13 +400,19 @@ def get_year(prs: PullRequests) -> int: return max(pr.created_at.year for pr in prs) -def get_branch_by_tag(tag: str) -> Optional[str]: - tag.removeprefix("v") +def get_branch_and_patch_by_tag(tag: str) -> Tuple[Optional[str], Optional[int]]: + tag = tag.removeprefix("v") versions = tag.split(".") - if len(versions) < 3: + if len(versions) < 4: print("ERROR: Can't get branch by tag") - return None - return f"{versions[0]}.{versions[1]}" + return None, None + try: + patch_version = int(versions[2]) + branch = f"{int(versions[0])}.{int(versions[1])}" + print(f"Branch [{branch}], patch version [{patch_version}]") + except ValueError: + return None, None + return branch, patch_version def main(): @@ -458,17 +464,22 @@ def main(): gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" - branch = get_branch_by_tag(TO_REF) - if branch and Shell.check(f"git show-ref --quiet {branch}"): - try: - if int(branch.split(".")[-1]) > 1: - query += f" base:{branch}" - print(f"NOTE: will use base branch to filter PRs {branch}") - except ValueError: - print(f"ERROR: cannot get minor version from branch {branch} - pass") - pass + + branch, patch = get_branch_and_patch_by_tag(TO_REF) + if branch and patch and Shell.check(f"git show-ref --quiet {branch}"): + if patch > 1: + query += f" base:{branch}" + print( + f"NOTE: It's a patch [{patch}]. will use base branch to filter PRs [{branch}]" + ) + else: + print( + f"NOTE: It's a first patch version. should count PRs merged on master - won't filter PRs by branch" + ) else: print(f"ERROR: invalid branch {branch} - pass") + + print(f"Fetch PRs with query {query}") prs = gh.get_pulls_from_search( query=query, merged=merged, sort="created", progress_func=tqdm.tqdm ) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 447aac74c7f..3182c0bc5d8 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -246,15 +246,14 @@ class Shell: @classmethod def check(cls, command): - result = subprocess.run( + proc = subprocess.Popen( command, shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=False, + stdout=subprocess.STDOUT, + stderr=subprocess.STDOUT, ) - return result.returncode == 0 + proc.wait() + return proc.returncode == 0 class Utils: From 4802ea540a4691c13386b74416352155b93f713d Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 11:57:54 +0200 Subject: [PATCH 341/496] improve ci_utils' Shell --- .github/workflows/pull_request.yml | 9 +++- .github/workflows/release.yml | 6 ++- pyproject.toml | 1 + tests/ci/artifactory.py | 84 +++++++++++------------------ tests/ci/auto_release.py | 5 +- tests/ci/ci_buddy.py | 6 ++- tests/ci/ci_definitions.py | 3 +- tests/ci/ci_utils.py | 72 ++++++++++++------------- tests/ci/create_release.py | 86 +++++++++++++++++------------- tests/ci/docker_images_helper.py | 8 +-- 10 files changed, 141 insertions(+), 139 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 854dff530e7..04bef1460a6 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -142,8 +142,13 @@ jobs: # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) Builds_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Builds') }} - needs: [RunConfig, StyleCheck, Builds_1, Builds_2] + if: ${{ !cancelled() + && needs.RunConfig.result == 'success' + && needs.StyleCheck.result != 'failure' + && needs.FastTest.result != 'failure' + && needs.BuildDockers.result != 'failure' + && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Builds') }} + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: Builds diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 8620d15ec19..7dc4e3298a6 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -55,12 +55,14 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push + SHA=$(git rev-list -n 1 "$GITHUB_TAG") + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - name: Check docker clickhouse/clickhouse-keeper building run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push + SHA=$(git rev-list -n 1 "$GITHUB_TAG") + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - name: Cleanup if: always() run: | diff --git a/pyproject.toml b/pyproject.toml index c89d46c0929..9bbeac3ddae 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -39,6 +39,7 @@ disable = ''' no-else-return, global-statement, f-string-without-interpolation, + consider-using-with, ''' [tool.pylint.SIMILARITIES] diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index a508374f856..71deaccf917 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -13,7 +13,6 @@ from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): - RCLONE = "rclone" S3FS = "s3fs" GEESEFS = "geesefs" @@ -31,9 +30,6 @@ class R2MountPoint: DEBUG = True # enable cache for mountpoint CACHE_ENABLED = False - # TODO: which mode is better: minimal/writes/full/off - _RCLONE_CACHE_MODE = "minimal" - UMASK = "0000" def __init__(self, app: str, dry_run: bool) -> None: assert app in MountPointApp @@ -59,18 +55,6 @@ class R2MountPoint: ) # without -o nomultipart there are errors like "Error 5 writing to /home/ubuntu/***.deb: Input/output error" self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" - elif self.app == MountPointApp.RCLONE: - # run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return - self.cache_dir = "/home/ubuntu/rclone_cache" - self.aux_mount_options += "--no-modtime " if self.NOMODTIME else "" - self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose - self.aux_mount_options += ( - f"--vfs-cache-mode {self._RCLONE_CACHE_MODE} --vfs-cache-max-size {self._CACHE_MAX_SIZE_GB}G" - if self.CACHE_ENABLED - else "--vfs-cache-mode off" - ) - # Use --no-modtime to try to avoid: ERROR : rpm/lts/clickhouse-client-24.3.6.5.x86_64.rpm: Failed to apply pending mod time - self.mount_cmd = f"rclone mount remote:{self.bucket_name} {self.MOUNT_POINT} --daemon --cache-dir {self.cache_dir} --umask 0000 --log-file {self.LOG_FILE} {self.aux_mount_options}" elif self.app == MountPointApp.GEESEFS: self.cache_dir = "/home/ubuntu/geesefs_cache" self.aux_mount_options += ( @@ -98,22 +82,17 @@ class R2MountPoint: ) _TEST_MOUNT_CMD = f"mount | grep -q {self.MOUNT_POINT}" - Shell.run(_CLEAN_LOG_FILE_CMD) - Shell.run(_UNMOUNT_CMD) - Shell.run(_MKDIR_CMD) - Shell.run(_MKDIR_FOR_CACHE) - if self.app != MountPointApp.RCLONE: - Shell.run(self.mount_cmd, check=True) - else: - # didn't manage to use simple run() and without blocking or failure - Shell.run_as_daemon(self.mount_cmd) + Shell.check(_CLEAN_LOG_FILE_CMD, verbose=True) + Shell.check(_UNMOUNT_CMD, verbose=True) + Shell.check(_MKDIR_CMD, verbose=True) + Shell.check(_MKDIR_FOR_CACHE, verbose=True) + Shell.check(self.mount_cmd, strict=True, verbose=True) time.sleep(3) - Shell.run(_TEST_MOUNT_CMD, check=True) + Shell.check(_TEST_MOUNT_CMD, strict=True, verbose=True) @classmethod def teardown(cls): - print(f"Unmount [{cls.MOUNT_POINT}]") - Shell.run(f"umount {cls.MOUNT_POINT}") + Shell.check(f"umount {cls.MOUNT_POINT}", verbose=True) class RepoCodenames(metaclass=WithIter): @@ -148,10 +127,9 @@ class DebianArtifactory: ] REPREPRO_CMD_PREFIX = f"reprepro --basedir {R2MountPoint.MOUNT_POINT}/configs/deb --outdir {R2MountPoint.MOUNT_POINT}/deb --verbose" cmd = f"{REPREPRO_CMD_PREFIX} includedeb {self.codename} {' '.join(paths)}" - print("Running export command:") - print(f" {cmd}") - Shell.run(cmd, check=True) - Shell.run("sync") + print("Running export commands:") + Shell.check(cmd, strict=True, verbose=True) + Shell.check("sync") if self.codename == RepoCodenames.LTS: packages_with_version = [ @@ -163,11 +141,11 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} copy {RepoCodenames.STABLE} {RepoCodenames.LTS} {' '.join(packages_with_version)}" print("Running copy command:") print(f" {cmd}") - Shell.run(cmd, check=True) - Shell.run("sync") + Shell.check(cmd, strict=True) + Shell.check("sync") def test_packages(self): - Shell.run("docker pull ubuntu:latest") + Shell.check("docker pull ubuntu:latest", strict=True) print(f"Test packages installation, version [{self.version}]") debian_command = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static={self.version} clickhouse-client={self.version}" cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' @@ -236,20 +214,18 @@ class RpmArtifactory: print(f"Exporting RPM packages into [{codename}]") for command in commands: - print("Running command:") - print(f" {command}") - Shell.run(command, check=True) + Shell.check(command, strict=True, verbose=True) update_public_key = f"gpg --armor --export {self._SIGN_KEY}" pub_key_path = dest_dir / "repodata" / "repomd.xml.key" print("Updating repomd.xml.key") - pub_key_path.write_text(Shell.run(update_public_key, check=True)) + pub_key_path.write_text(Shell.get_output_or_raise(update_public_key)) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - Shell.run("sync") + Shell.check("sync") def test_packages(self): - Shell.run("docker pull fedora:latest") + Shell.check("docker pull fedora:latest", strict=True) print(f"Test package installation, version [{self.version}]") rpm_command = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1" cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' @@ -302,26 +278,30 @@ class TgzArtifactory: if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - Shell.run("sync") + Shell.check("sync") def test_packages(self): tgz_file = "/tmp/tmp.tgz" tgz_sha_file = "/tmp/tmp.tgz.sha512" cmd = f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" - Shell.run( + Shell.check( cmd, - check=True, + strict=True, + verbose=True, ) - Shell.run( + Shell.check( f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512", - check=True, + strict=True, + verbose=True, + ) + expected_checksum = Shell.get_output_or_raise(f"cut -d ' ' -f 1 {tgz_sha_file}") + actual_checksum = Shell.get_output_or_raise( + f"sha512sum {tgz_file} | cut -d ' ' -f 1" ) - expected_checksum = Shell.run(f"cut -d ' ' -f 1 {tgz_sha_file}", check=True) - actual_checksum = Shell.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") assert ( expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" - Shell.run("rm /tmp/tmp.tgz*") + Shell.check("rm /tmp/tmp.tgz*", verbose=True) self.release_info.tgz_command = cmd self.release_info.dump() @@ -373,9 +353,9 @@ if __name__ == "__main__": args = parse_args() """ - Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: - ERROR : IO error: NotImplemented: versionId not implemented - Failed to copy: NotImplemented: versionId not implemented + S3FS - very slow with a big repo + RCLONE - fuse had many different errors with r2 remote and completely removed + GEESEFS ? """ mp = R2MountPoint(MountPointApp.GEESEFS, dry_run=args.dry_run) if args.export_debian: diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index f2386fe207f..6c17b4c74ad 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -85,7 +85,7 @@ class AutoReleaseInfo: def _prepare(token): assert len(token) > 10 os.environ["GH_TOKEN"] = token - Shell.run("gh auth status", check=True) + Shell.check("gh auth status") gh = GitHub(token) prs = gh.get_release_pulls(GITHUB_REPOSITORY) @@ -106,9 +106,8 @@ def _prepare(token): latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - commits = Shell.run( + commits = Shell.get_output_or_raise( f"git rev-list --first-parent {latest_release_tag.tag}..origin/{pr.head.ref}", - check=True, ).split("\n") commit_num = len(commits) print( diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index dfb5885270a..138909c1db0 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -120,8 +120,10 @@ class CIBuddy: ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: - instance_id = Shell.run("ec2metadata --instance-id") or instance_id - instance_type = Shell.run("ec2metadata --instance-type") or instance_type + instance_id = Shell.get_output("ec2metadata --instance-id") or instance_id + instance_type = ( + Shell.get_output("ec2metadata --instance-type") or instance_type + ) if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") sign = ":red_circle:" if not critical else ":black_circle:" diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 054b554b8fa..51de8c63509 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -554,7 +554,7 @@ class CommonJobConfigs: run_command="sqllogic_test.py", timeout=10800, release_only=True, - runner_type=Runners.STYLE_CHECKER, + runner_type=Runners.FUNC_TESTER, ) SQL_TEST = JobConfig( job_name_keyword="sqltest", @@ -582,6 +582,7 @@ class CommonJobConfigs: digest=DigestConfig( include_paths=[ "tests/ci/docker_server.py", + "tests/ci/docker_images_helper.py", "./docker/server", ] ), diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 3182c0bc5d8..cd21554788c 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -2,6 +2,7 @@ import json import os import re import subprocess +import sys import time from contextlib import contextmanager from pathlib import Path @@ -192,7 +193,7 @@ class GHActions: get_url_cmd = ( f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" ) - url = Shell.run(get_url_cmd) + url = Shell.get_output(get_url_cmd) if not url: print(f"ERROR: PR nor found, branch [{branch}]") return url @@ -200,59 +201,56 @@ class GHActions: class Shell: @classmethod - def run_strict(cls, command): + def get_output_or_raise(cls, command): + return cls.get_output(command, strict=True) + + @classmethod + def get_output(cls, command, strict=False): res = subprocess.run( command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE, text=True, - check=True, + check=strict, ) return res.stdout.strip() @classmethod - def run(cls, command, check=False, dry_run=False, **kwargs): + def check( + cls, + command, + strict=False, + verbose=False, + dry_run=False, + stdin_str=None, + **kwargs, + ): if dry_run: print(f"Dry-ryn. Would run command [{command}]") - return "" - print(f"Run command [{command}]") - res = "" - result = subprocess.run( - command, - shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=False, - **kwargs, - ) - if result.returncode == 0: - print(f"stdout: {result.stdout.strip()}") - res = result.stdout - else: - print( - f"ERROR: stdout: {result.stdout.strip()}, stderr: {result.stderr.strip()}" - ) - if check: - assert result.returncode == 0 - return res.strip() - - @classmethod - def run_as_daemon(cls, command): - print(f"Run daemon command [{command}]") - subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with - return 0, "" - - @classmethod - def check(cls, command): + return 0 + if verbose: + print(f"Run command [{command}]") proc = subprocess.Popen( command, shell=True, - stdout=subprocess.STDOUT, stderr=subprocess.STDOUT, + stdout=subprocess.PIPE, + stdin=subprocess.PIPE if stdin_str else None, + universal_newlines=True, + start_new_session=True, + bufsize=1, + errors="backslashreplace", + **kwargs, ) + if stdin_str: + proc.communicate(input=stdin_str) + elif proc.stdout: + for line in proc.stdout: + sys.stdout.write(line) proc.wait() + if strict: + assert proc.returncode == 0 return proc.returncode == 0 @@ -277,7 +275,7 @@ class Utils: @staticmethod def clear_dmesg(): - Shell.run("sudo dmesg --clear ||:") + Shell.check("sudo dmesg --clear", verbose=True) @staticmethod def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index b02a0bb8ed5..0d505d6ccc7 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -137,12 +137,13 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - Shell.run( + Shell.check( f"git merge-base --is-ancestor {commit_ref} origin/master", - check=True, + strict=True, + verbose=True, ) with checkout(commit_ref): - commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) + commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -154,13 +155,13 @@ class ReleaseInfo: ), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]" release_tag = version.describe previous_release_tag = expected_prev_tag - previous_release_sha = Shell.run_strict( + previous_release_sha = Shell.get_output_or_raise( f"git rev-parse {previous_release_tag}" ) assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) + commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -168,11 +169,16 @@ class ReleaseInfo: version.with_description(codename) release_branch = f"{version.major}.{version.minor}" release_tag = version.describe - Shell.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags", check=True) + Shell.check( + f"{GIT_PREFIX} fetch origin {release_branch} --tags", + strict=True, + verbose=True, + ) # check commit is right and on a right branch - Shell.run( + Shell.check( f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}", - check=True, + strict=True, + verbose=True, ) if version.patch == 1: expected_version = copy(version) @@ -197,7 +203,7 @@ class ReleaseInfo: False ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]" - previous_release_sha = Shell.run_strict( + previous_release_sha = Shell.get_output_or_raise( f"git rev-parse {previous_release_tag}" ) assert previous_release_sha @@ -226,25 +232,26 @@ class ReleaseInfo: def push_release_tag(self, dry_run: bool) -> None: if dry_run: # remove locally created tag from prev run - Shell.run( - f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag} ||:" + Shell.check( + f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag}" ) # Create release tag print( f"Create and push release tag [{self.release_tag}], commit [{self.commit_sha}]" ) tag_message = f"Release {self.release_tag}" - Shell.run( + Shell.check( f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}", - check=True, + strict=True, + verbose=True, ) cmd_push_tag = f"{GIT_PREFIX} push origin {self.release_tag}:{self.release_tag}" - Shell.run(cmd_push_tag, dry_run=dry_run, check=True) + Shell.check(cmd_push_tag, dry_run=dry_run, strict=True, verbose=True) @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - Shell.run(cmd, dry_run=dry_run, check=True) + Shell.check(cmd, dry_run=dry_run, strict=True) def push_new_release_branch(self, dry_run: bool) -> None: assert ( @@ -261,7 +268,7 @@ class ReleaseInfo: ), f"Unexpected current version in git, must precede [{self.version}] by one step, actual [{version.string}]" if dry_run: # remove locally created branch from prev run - Shell.run( + Shell.check( f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch}" ) print( @@ -275,7 +282,7 @@ class ReleaseInfo: cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) - Shell.run(cmd_push_branch, dry_run=dry_run, check=True) + Shell.check(cmd_push_branch, dry_run=dry_run, strict=True, verbose=True) print("Create and push backport tags for new release branch") ReleaseInfo._create_gh_label( @@ -284,13 +291,14 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - Shell.run( + Shell.check( f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, dry_run=dry_run, - check=True, + strict=True, + verbose=True, ) def update_version_and_contributors_list(self, dry_run: bool) -> None: @@ -316,13 +324,19 @@ class ReleaseInfo: body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" - Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) - Shell.run(cmd_push_branch, check=True, dry_run=dry_run) - Shell.run(cmd_create_pr, check=True, dry_run=dry_run) + Shell.check( + cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) + Shell.check(cmd_create_pr, strict=True, dry_run=dry_run, verbose=True) if dry_run: - Shell.run(f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'") - Shell.run( - f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, ) self.version_bump_pr = "dry-run" else: @@ -358,7 +372,7 @@ class ReleaseInfo: cmds.append(f"gh release upload {self.release_tag} {file}") if not dry_run: for cmd in cmds: - Shell.run(cmd, check=True) + Shell.check(cmd, strict=True, verbose=True) self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" else: print("Dry-run, would run commands:") @@ -424,7 +438,7 @@ class PackageDownloader: self.macos_package_files = ["clickhouse-macos", "clickhouse-macos-aarch64"] self.file_to_type = {} - Shell.run(f"mkdir -p {self.LOCAL_DIR}") + Shell.check(f"mkdir -p {self.LOCAL_DIR}") for package_type in self.PACKAGE_TYPES: for package in self.package_names: @@ -474,7 +488,7 @@ class PackageDownloader: return res def run(self): - Shell.run(f"rm -rf {self.LOCAL_DIR}/*") + Shell.check(f"rm -rf {self.LOCAL_DIR}/*") for package_file in ( self.deb_package_files + self.rpm_package_files + self.tgz_package_files ): @@ -549,33 +563,33 @@ class PackageDownloader: @contextmanager def checkout(ref: str) -> Iterator[None]: - orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + orig_ref = Shell.get_output_or_raise(f"{GIT_PREFIX} symbolic-ref --short HEAD") rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref if ref not in (orig_ref,): - Shell.run(f"{GIT_PREFIX} checkout {ref}") + Shell.check(f"{GIT_PREFIX} checkout {ref}", strict=True, verbose=True) try: yield except (Exception, KeyboardInterrupt) as e: print(f"ERROR: Exception [{e}]") - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) raise - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) @contextmanager def checkout_new(ref: str) -> Iterator[None]: - orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + orig_ref = Shell.get_output_or_raise(f"{GIT_PREFIX} symbolic-ref --short HEAD") rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref - Shell.run(f"{GIT_PREFIX} checkout -b {ref}", check=True) + Shell.check(f"{GIT_PREFIX} checkout -b {ref}", strict=True, verbose=True) try: yield except (Exception, KeyboardInterrupt) as e: print(f"ERROR: Exception [{e}]") - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) raise - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) def parse_args() -> argparse.Namespace: diff --git a/tests/ci/docker_images_helper.py b/tests/ci/docker_images_helper.py index e6869852c4e..f0323145cfa 100644 --- a/tests/ci/docker_images_helper.py +++ b/tests/ci/docker_images_helper.py @@ -19,11 +19,11 @@ def docker_login(relogin: bool = True) -> None: if relogin or not Shell.check( "docker system info | grep --quiet -E 'Username|Registry'" ): - Shell.run( # pylint: disable=unexpected-keyword-arg + Shell.check( # pylint: disable=unexpected-keyword-arg "docker login --username 'robotclickhouse' --password-stdin", - input=get_parameter_from_ssm("dockerhub_robot_password"), + strict=True, + stdin_str=get_parameter_from_ssm("dockerhub_robot_password"), encoding="utf-8", - check=True, ) @@ -42,7 +42,7 @@ class DockerImage: def pull_image(image: DockerImage) -> DockerImage: try: logging.info("Pulling image %s - start", image) - Shell.run(f"docker pull {image}", check=True) + Shell.check(f"docker pull {image}", strict=True) logging.info("Pulling image %s - done", image) except Exception as ex: logging.info("Got exception pulling docker %s", ex) From a6d0b7afbb8299eb8cf056368e93267ef51359ba Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 18:05:19 +0200 Subject: [PATCH 342/496] recovery option --- .github/workflows/create_release.yml | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index c3126abe461..424dfe60be4 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -16,6 +16,11 @@ concurrency: options: - patch - new + only-repo: + description: 'Run only repos updates including docker (recovery)' + required: false + default: false + type: boolean dry-run: description: 'Dry run' required: false @@ -54,11 +59,12 @@ jobs: run: | python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push Git Tag for the Release + if: ${{ ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} + if: ${{ inputs.type == 'new' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} @@ -67,7 +73,7 @@ jobs: # run: | # python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" @@ -87,7 +93,7 @@ jobs: python3 ./utils/security-generator/generate_security.py > SECURITY.md git diff HEAD - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + if: ${{ inputs.type == 'patch' && ! inputs.dry-run && ! inputs.only-repo }} uses: peter-evans/create-pull-request@v6 with: author: "robot-clickhouse " @@ -105,7 +111,7 @@ jobs: ### Changelog category (leave one): - Not for changelog (changelog entry is not required) - name: Complete previous steps and Restore git state - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-completed From dab5eb9c24cc2f43a0ad8ee65ecac613896cff10 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Aug 2024 16:16:34 +0000 Subject: [PATCH 343/496] Fix INTERPOLATE by constant. Fix other tests. --- src/Analyzer/InterpolateNode.cpp | 2 +- src/Analyzer/InterpolateNode.h | 2 ++ src/Analyzer/Resolve/QueryAnalyzer.cpp | 13 ++++------- src/Planner/CollectTableExpressionData.cpp | 2 +- src/Planner/Planner.cpp | 22 +++++++++++++++++++ src/Planner/PlannerExpressionAnalysis.cpp | 3 +++ src/Processors/QueryPlan/FillingStep.cpp | 14 +++++++++++- ..._no_aggregates_and_constant_keys.reference | 4 ++-- ...15_analyzer_materialized_constants_bug.sql | 2 +- 9 files changed, 49 insertions(+), 15 deletions(-) diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 97dc79f565b..17c734cf386 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -24,7 +24,7 @@ void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_st { buffer << std::string(indent, ' ') << "INTERPOLATE id: " << format_state.getNodeId(this); - buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << expression_name << " \n"; getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); buffer << '\n' << std::string(indent + 2, ' ') << "INTERPOLATE_EXPRESSION\n"; diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index ec493ed8bdd..eb3d64d7170 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -50,6 +50,8 @@ public: return QueryTreeNodeType::INTERPOLATE; } + const std::string & getExpressionName() const { return expression_name; } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; protected: diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..e973bd8fb34 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -64,6 +64,8 @@ #include #include +#include + #include namespace ProfileEvents @@ -4122,11 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo { auto & interpolate_node_typed = interpolate_node->as(); - auto * column_to_interpolate = interpolate_node_typed.getExpression()->as(); - if (!column_to_interpolate) - throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found", - interpolate_node_typed.getExpression()->formatASTForErrorMessage()); - auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName(); + auto column_to_interpolate_name = interpolate_node_typed.getExpressionName(); resolveExpressionNode(interpolate_node_typed.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -4135,14 +4133,11 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo auto & interpolation_to_resolve = interpolate_node_typed.getInterpolateExpression(); IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/); - auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node_typed.getExpression()); + auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node); if (is_column_constant) interpolate_scope.expression_argument_name_to_node.emplace(column_to_interpolate_name, fake_column_node); resolveExpressionNode(interpolation_to_resolve, interpolate_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - if (is_column_constant) - interpolation_to_resolve = interpolation_to_resolve->cloneAndReplace(fake_column_node, interpolate_node_typed.getExpression()); } } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 2fe62aa9be0..c48813a4ed4 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -46,7 +46,7 @@ public: auto column_source_node = column_node->getColumnSource(); auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::LAMBDA) + if (column_source_node_type == QueryTreeNodeType::LAMBDA || column_source_node_type == QueryTreeNodeType::INTERPOLATE) return; /// JOIN using expression diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 968642dc9de..b837d9428a1 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -744,6 +744,8 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, } else { + ActionsDAG rename_dag; + for (auto & interpolate_node : interpolate_list_nodes) { auto & interpolate_node_typed = interpolate_node->as(); @@ -772,8 +774,28 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name); interpolate_actions_dag.getOutputs().push_back(alias_node); + + /// Here we fix INTERPOLATE by constant expression. + /// Example from 02336_sort_optimization_with_fill: + /// + /// SELECT 5 AS x, 'Hello' AS s ORDER BY x WITH FILL FROM 1 TO 10 INTERPOLATE (s AS s||'A') + /// + /// For this query, INTERPOLATE_EXPRESSION would be : s AS concat(s, 'A'), + /// so that interpolate_actions_dag would have INPUT `s`. + /// + /// However, INPUT `s` does not exist. Instead, we have a constant with execution name 'Hello'_String. + /// To fix this, we prepend a rename : 'Hello'_String -> s + if (const auto * constant_node = interpolate_node_typed.getExpression()->as()) + { + const auto * node = &rename_dag.addInput(alias_node->result_name, alias_node->result_type); + node = &rename_dag.addAlias(*node, interpolate_node_typed.getExpressionName()); + rename_dag.getOutputs().push_back(node); + } } + if (!rename_dag.getOutputs().empty()) + interpolate_actions_dag = ActionsDAG::merge(std::move(rename_dag), std::move(interpolate_actions_dag)); + interpolate_actions_dag.removeUnusedActions(); } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 2b67c96d843..ed3f78193ee 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -462,6 +462,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); + if (interpolate_node_typed.getExpression()->getNodeType() == QueryTreeNodeType::CONSTANT) + continue; + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 81622389ada..8687886447a 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include namespace DB @@ -58,14 +59,25 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build void FillingStep::describeActions(FormatSettings & settings) const { - settings.out << String(settings.offset, ' '); + String prefix(settings.offset, settings.indent_char); + settings.out << prefix; dumpSortDescription(sort_description, settings.out); settings.out << '\n'; + if (interpolate_description) + { + auto expression = std::make_shared(interpolate_description->actions.clone()); + expression->describeActions(settings.out, prefix); + } } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description)); + if (interpolate_description) + { + auto expression = std::make_shared(interpolate_description->actions.clone()); + map.add("Expression", expression->toTree()); + } } void FillingStep::updateOutputStream() diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index 63b8a9d14fc..fc77ed8a241 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -8,13 +8,13 @@ 40 41 -0 +41 2 42 2 42 43 -0 +43 11 11 diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql index f9ec28d09d8..b2fd69d75d0 100644 --- a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql @@ -20,7 +20,7 @@ WITH ( SELECT coalesce(materialize(toLowCardinality(toNullable(1))), 10, NULL), max(v) -FROM remote('127.0.0.{1,2}', default, test__fuzz_21) +FROM remote('127.0.0.{1,2}', currentDatabase(), test__fuzz_21) GROUP BY coalesce(NULL), coalesce(1, 10, 10, materialize(NULL)); From d683fb05a009ed3f58c0e11fc329c3783f934369 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Aug 2024 19:05:30 +0200 Subject: [PATCH 344/496] Fix --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 ++++++--- src/Interpreters/Cache/FileCache.cpp | 35 ++++++++++++------- src/Interpreters/Cache/FileCache.h | 4 ++- src/Interpreters/Cache/FileSegment.cpp | 9 ++++- src/Interpreters/Cache/FileSegment.h | 4 ++- tests/config/config.d/storage_conf.xml | 3 +- 6 files changed, 51 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index c928d25c7b8..b471f3fc58f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -135,8 +135,11 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet( + cache_key, file_offset_of_buffer_end, size, file_size.value(), + create_settings, settings.filesystem_cache_segments_batch_size, user); } + return !file_segments->empty(); } @@ -158,8 +161,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current offset: {}", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + "Having {} file segments to read: {}, current read range: [{}, {})", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); initialized = true; } @@ -1043,6 +1046,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; + + LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", + remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); + size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1055,8 +1062,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {})", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); + fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); } swap(*implementation_buffer); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index a88c0de2cfe..0a03f5dcc7d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -316,14 +316,14 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: return result; } -std::vector FileCache::splitRange(size_t offset, size_t size) +std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { assert(size > 0); std::vector ranges; size_t current_pos = offset; size_t end_pos_non_included = offset + size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -343,17 +343,20 @@ FileSegments FileCache::splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + /// We take `size` as a soft limit and `aligned_size` as a hard limit. auto current_pos = offset; auto end_pos_non_included = offset + size; size_t current_file_segment_size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -369,6 +372,8 @@ FileSegments FileCache::splitRangeIntoFileSegments( current_pos += current_file_segment_size; } + chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), + fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); return file_segments; } @@ -376,6 +381,7 @@ void FileCache::fillHolesWithEmptyFileSegments( LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & create_settings) @@ -442,7 +448,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, hole_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -479,7 +485,7 @@ void FileCache::fillHolesWithEmptyFileSegments( chassert(!file_segments_limit || file_segments.size() < file_segments_limit); - if (current_pos <= range.right) + if (current_pos <= non_aligned_right_offset) { /// ________] -- requested range /// _____] @@ -487,6 +493,7 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; + auto non_aligned_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { @@ -497,7 +504,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, non_aligned_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -542,7 +549,7 @@ FileSegmentsHolderPtr FileCache::set( else { file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -659,9 +666,13 @@ FileCache::getOrSet( } } + chassert(range.left >= aligned_offset); + if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, create_settings); + file_segments = splitRangeIntoFileSegments( + *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), + FileSegment::State::EMPTY, file_segments_limit, create_settings); } else { @@ -669,9 +680,9 @@ FileCache::getOrSet( chassert(file_segments.back()->range().left <= range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(offset)) + if (!file_segments.front()->range().contains(range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", @@ -713,7 +724,7 @@ FileSegmentsHolderPtr FileCache::get( } fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 527fd9d5edf..3f7eec73b56 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -263,7 +263,7 @@ private: /// Split range into subranges by max_file_segment_size, /// each subrange size must be less or equal to max_file_segment_size. - std::vector splitRange(size_t offset, size_t size); + std::vector splitRange(size_t offset, size_t size, size_t aligned_size); /// Split range into subranges by max_file_segment_size (same as in splitRange()) /// and create a new file segment for each subrange. @@ -273,6 +273,7 @@ private: LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); @@ -281,6 +282,7 @@ private: LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 1664a91b694..c46fb978ae4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1008,7 +1008,12 @@ FileSegment & FileSegmentsHolder::add(FileSegmentPtr && file_segment) return *file_segments.back(); } -String FileSegmentsHolder::toString() +String FileSegmentsHolder::toString(bool with_state) +{ + return DB::toString(file_segments, with_state); +} + +String toString(const FileSegments & file_segments, bool with_state) { String ranges; for (const auto & file_segment : file_segments) @@ -1018,6 +1023,8 @@ String FileSegmentsHolder::toString() ranges += file_segment->range().toString(); if (file_segment->isUnbound()) ranges += "(unbound)"; + if (with_state) + ranges += "(" + FileSegment::stateToString(file_segment->state()) + ")"; } return ranges; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index d6b37b60dc1..25ffb880b45 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -291,7 +291,7 @@ struct FileSegmentsHolder : private boost::noncopyable size_t size() const { return file_segments.size(); } - String toString(); + String toString(bool with_state = false); void popFront() { completeAndPopFrontImpl(); } @@ -317,4 +317,6 @@ private: using FileSegmentsHolderPtr = std::unique_ptr; +String toString(const FileSegments & file_segments, bool with_state = false); + } diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 7a9b579c00a..4daa64b520d 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,8 @@ cache s3_disk s3_cache/ - 104857600 + 100Mi + 5Mi 1 100 LRU From 1e8d0d4a5e8d83a1d123a4b5b6c5a91b41caac1c Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 31 Jul 2024 06:09:14 +0000 Subject: [PATCH 345/496] disable parallel run for network_receive_time_metric_insert If run in parallel, several tests may affect the value of the `NetworkReceiveElapsedMicroseconds` profile event. This may contribute to test flakiness. --- .../0_stateless/01923_network_receive_time_metric_insert.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index 97835d97965..77b909ed89e 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel # Tag no-fasttest: needs pv +# Tag no-parallel: reads from a system table CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4709222dd1f3a37c5f97e638526c21ade6b5218f Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 31 Jul 2024 23:16:43 +0000 Subject: [PATCH 346/496] print debug info if the test fails --- ...1923_network_receive_time_metric_insert.sh | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index 77b909ed89e..adf4fd96a00 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -13,9 +13,23 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) seq 1 1000 | pv --quiet --rate-limit 400 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" # We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client. -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; - WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS time - SELECT time >= 1000000 ? 1 : time FROM system.query_log - WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" +result=$(${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; + WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS elapsed_us + SELECT elapsed_us FROM system.query_log + WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 'QueryFinish' + ORDER BY event_time DESC LIMIT 1;") + +elapsed_us=$(echo $result | sed 's/ .*//') + +min_elapsed_us=1000000 +if [[ "$elapsed_us" -ge "$min_elapsed_us" ]]; then + echo 1 +else + # Print debug info + ${CLICKHOUSE_CLIENT} --query " + WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS elapsed_us + SELECT query_start_time_microseconds, event_time_microseconds, query_duration_ms, elapsed_us, query FROM system.query_log + WHERE current_database = currentDatabase() and event_date >= yesterday() AND type = 'QueryFinish' ORDER BY query_start_time;" +fi ${CLICKHOUSE_CLIENT} --query "DROP TABLE t" From fc72742e37ae78f47b3e55a969e088f5c372ee36 Mon Sep 17 00:00:00 2001 From: sakulali Date: Fri, 2 Aug 2024 02:00:20 +0800 Subject: [PATCH 347/496] ping CI From 30e0c1a1b8479e9b6be0701ba21e6050906a7e43 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 18:46:09 +0200 Subject: [PATCH 348/496] try less mem for geesefs --- .github/workflows/create_release.yml | 21 +++++++++++---------- tests/ci/artifactory.py | 8 +++++--- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 424dfe60be4..3c61fa4cfe1 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -17,7 +17,7 @@ concurrency: - patch - new only-repo: - description: 'Run only repos updates including docker (recovery)' + description: 'Run only repos updates including docker (repo-recovery, tests)' required: false default: false type: boolean @@ -68,10 +68,11 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} -# - name: Bump CH Version and Update Contributors' List -# shell: bash -# run: | -# python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + if: ${{ ! inputs.only-repo }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash @@ -117,11 +118,11 @@ jobs: python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" -# - name: Create GH Release -# shell: bash -# if: ${{ inputs.type == 'patch' }} -# run: | -# python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Create GH Release + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 71deaccf917..8bba7bca30e 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -67,8 +67,8 @@ class R2MountPoint: f" --shared-config=/home/ubuntu/.r2_auth_test " ) if self.DEBUG: - self.aux_mount_options += " --debug_s3 --debug_fuse " - self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=2050 --gc-interval=100 --max-flushers=5 --max-parallel-parts=1 --max-parallel-copy=2 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" + self.aux_mount_options += " --debug_s3 " + self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=1000 --gc-interval=100 --max-flushers=10 --max-parallel-parts=1 --max-parallel-copy=10 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" else: assert False @@ -207,8 +207,10 @@ class RpmArtifactory: for package in paths: _copy_if_not_exists(Path(package), dest_dir) + # switching between different fuse providers invalidates --update option (apparently some fuse(s) can mess around with mtime) + # add --skip-stat to skip mtime check commands = ( - f"createrepo_c --local-sqlite --workers=2 --update --verbose {dest_dir}", + f"createrepo_c --local-sqlite --workers=2 --update --skip-stat --verbose {dest_dir}", f"gpg --sign-with {self._SIGN_KEY} --detach-sign --batch --yes --armor {dest_dir / 'repodata' / 'repomd.xml'}", ) print(f"Exporting RPM packages into [{codename}]") From e034558f74a1cd46bb8fbdfac3b7dc6d25165f4e Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 20:51:36 +0200 Subject: [PATCH 349/496] add automerge prs step --- .github/workflows/create_release.yml | 4 +++ pyproject.toml | 1 + tests/ci/ci_utils.py | 2 +- tests/ci/create_release.py | 47 +++++++++++++++++++++++++++- tests/ci/release.py | 1 + 5 files changed, 53 insertions(+), 2 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 3c61fa4cfe1..e27db1b09a4 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -171,6 +171,10 @@ jobs: export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed + - name: Update release info. Merge created PRs + shell: bash + run: | + python3 ./tests/ci/create_release.py --merge-prs ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Set current Release progress to Completed with OK shell: bash run: | diff --git a/pyproject.toml b/pyproject.toml index 9bbeac3ddae..4268901e7f9 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -40,6 +40,7 @@ disable = ''' global-statement, f-string-without-interpolation, consider-using-with, + use-maxsplit-arg, ''' [tool.pylint.SIMILARITIES] diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index cd21554788c..4f696a2c55a 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -228,7 +228,7 @@ class Shell: ): if dry_run: print(f"Dry-ryn. Would run command [{command}]") - return 0 + return True if verbose: print(f"Run command [{command}]") proc = subprocess.Popen( diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 0d505d6ccc7..c407a74fbf0 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -43,6 +43,7 @@ class ReleaseProgress: TEST_TGZ = "test TGZ packages" TEST_RPM = "test RPM packages" TEST_DEB = "test DEB packages" + MERGE_CREATED_PRS = "merge created PRs" COMPLETED = "completed" @@ -101,6 +102,7 @@ class ReleaseInfo: previous_release_sha: str changelog_pr: str = "" version_bump_pr: str = "" + prs_merged: bool = False release_url: str = "" debian_command: str = "" rpm_command: str = "" @@ -380,6 +382,38 @@ class ReleaseInfo: self.release_url = f"dry-run" self.dump() + def merge_prs(self, dry_run: bool) -> None: + repo = CI.Envs.GITHUB_REPOSITORY + assert self.version_bump_pr + if dry_run: + version_bump_pr_num = 12345 + else: + version_bump_pr_num = int(self.version_bump_pr.split("/")[-1]) + print("Merging Version bump PR") + res_1 = Shell.check( + f"gh pr merge {version_bump_pr_num} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + + res_2 = True + if not self.release_tag.endswith("-new"): + assert self.changelog_pr + print("Merging ChangeLog PR") + if dry_run: + changelog_pr_num = 23456 + else: + changelog_pr_num = int(self.changelog_pr.split("/")[-1]) + res_2 = Shell.check( + f"gh pr merge {changelog_pr_num} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + else: + assert not self.changelog_pr + + self.prs_merged = res_1 and res_2 + class RepoTypes: RPM = "rpm" @@ -627,6 +661,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Create GH Release object and attach all packages", ) + parser.add_argument( + "--merge-prs", + action="store_true", + help="Merge PRs with version, changelog updates", + ) parser.add_argument( "--post-status", action="store_true", @@ -732,7 +771,6 @@ if __name__ == "__main__": if args.post_status: release_info = ReleaseInfo.from_file() - release_info.update_release_info(dry_run=args.dry_run) if release_info.is_new_release_branch(): title = "New release branch" else: @@ -766,6 +804,13 @@ if __name__ == "__main__": ri.progress_description = ReleaseProgressDescription.OK ri.dump() + if args.merge_prs: + with ReleaseContextManager( + release_progress=ReleaseProgress.MERGE_CREATED_PRS + ) as release_info: + release_info.update_release_info(dry_run=args.dry_run) + release_info.merge_prs(dry_run=args.dry_run) + # tear down ssh if _ssh_agent and _key_pub: _ssh_agent.remove(_key_pub) diff --git a/tests/ci/release.py b/tests/ci/release.py index 2de20d00a00..b26d6205f3b 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -689,4 +689,5 @@ def main(): if __name__ == "__main__": + assert False, "Script Deprecated, ask ci team for help" main() From 67b11300e45f6e24c3515a978d23a9bc998a666e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 1 Aug 2024 21:05:49 +0000 Subject: [PATCH 350/496] Fix crash in KeyCondition::cloneASTWithInversionPushDown() caused by type change --- src/Storages/MergeTree/KeyCondition.cpp | 82 +++++++++---------- .../03215_key_condition_bug.reference | 1 + .../0_stateless/03215_key_condition_bug.sql | 3 + 3 files changed, 44 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/03215_key_condition_bug.reference create mode 100644 tests/queries/0_stateless/03215_key_condition_bug.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 69bffac9160..eaf9f0af623 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -566,6 +566,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } const ActionsDAG::Node * res = nullptr; + bool handled_inversion = false; switch (node.type) { @@ -582,7 +583,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( /// Re-generate column name for constant. /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. /// DAG from PK does not use it. This breaks matching by column name sometimes. - /// Ideally, we should not compare manes, but DAG subtrees instead. + /// Ideally, we should not compare names, but DAG subtrees instead. name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); else name = node.result_name; @@ -593,9 +594,9 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( case (ActionsDAG::ActionType::ALIAS): { /// Ignore aliases - const auto & alias = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); - to_inverted[&node] = &alias; - return alias; + res = &cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); + handled_inversion = true; + break; } case (ActionsDAG::ActionType::ARRAY_JOIN): { @@ -608,20 +609,10 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( auto name = node.function_base->getName(); if (name == "not") { - const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion); - to_inverted[&node] = &arg; - return arg; + res = &cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion); + handled_inversion = true; } - - if (name == "materialize") - { - /// Ignore materialize - const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); - to_inverted[&node] = &arg; - return arg; - } - - if (name == "indexHint") + else if (name == "indexHint") { ActionsDAG::NodeRawConstPtrs children; if (const auto * adaptor = typeid_cast(node.function_base.get())) @@ -636,12 +627,10 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - const auto & func = inverted_dag.addFunction(node.function_base, children, ""); - to_inverted[&node] = &func; - return func; + res = &inverted_dag.addFunction(node.function_base, children, ""); + handled_inversion = true; } - - if (need_inversion && (name == "and" || name == "or")) + else if (need_inversion && (name == "and" || name == "or")) { ActionsDAG::NodeRawConstPtrs children(node.children); @@ -659,34 +648,43 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( /// We match columns by name, so it is important to fill name correctly. /// So, use empty string to make it automatically. - const auto & func = inverted_dag.addFunction(function_builder, children, ""); - to_inverted[&node] = &func; - return func; + res = &inverted_dag.addFunction(function_builder, children, ""); + handled_inversion = true; } - - ActionsDAG::NodeRawConstPtrs children(node.children); - - for (auto & arg : children) - arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false); - - auto it = inverse_relations.find(name); - if (it != inverse_relations.end()) + else { - const auto & func_name = need_inversion ? it->second : it->first; - auto function_builder = FunctionFactory::instance().get(func_name, context); - const auto & func = inverted_dag.addFunction(function_builder, children, ""); - to_inverted[&node] = &func; - return func; - } + ActionsDAG::NodeRawConstPtrs children(node.children); - res = &inverted_dag.addFunction(node.function_base, children, ""); - chassert(res->result_type == node.result_type); + for (auto & arg : children) + arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false); + + auto it = inverse_relations.find(name); + if (it != inverse_relations.end()) + { + const auto & func_name = need_inversion ? it->second : it->first; + auto function_builder = FunctionFactory::instance().get(func_name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); + handled_inversion = true; + } + else + { + res = &inverted_dag.addFunction(node.function_base, children, ""); + chassert(res->result_type == node.result_type); + } + } } } - if (need_inversion) + if (!handled_inversion && need_inversion) res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, ""); + /// Make sure we don't change any data types (e.g. remove LowCardinality). + /// If it turns out that we actually want to change data types sometimes, it's ok to remove this + /// check *and* replace all `addFunction(node.function_base, ...)` calls above with + /// `addFunction(FunctionFactory::instance().get(name, context), ...)` to re-resolve overloads. + if (!node.result_type->equals(*res->result_type)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", node.result_type->getName(), res->result_type->getName(), node.result_name); + to_inverted[&node] = res; return *res; } diff --git a/tests/queries/0_stateless/03215_key_condition_bug.reference b/tests/queries/0_stateless/03215_key_condition_bug.reference new file mode 100644 index 00000000000..84ab67a85e0 --- /dev/null +++ b/tests/queries/0_stateless/03215_key_condition_bug.reference @@ -0,0 +1 @@ +(0) diff --git a/tests/queries/0_stateless/03215_key_condition_bug.sql b/tests/queries/0_stateless/03215_key_condition_bug.sql new file mode 100644 index 00000000000..ef2113e81f8 --- /dev/null +++ b/tests/queries/0_stateless/03215_key_condition_bug.sql @@ -0,0 +1,3 @@ +CREATE TABLE t (x Int8) ENGINE MergeTree ORDER BY x; +INSERT INTO t VALUES (1); +SELECT arrayJoin([tuple((toNullable(10) * toLowCardinality(20)) < materialize(30))]) AS row FROM t WHERE row.1 = 0; \ No newline at end of file From eac2c9fc3d8a88c1033e0f23e048421ecf4db850 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 351/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 69bd306a445a6bc8a55be14bb0080864921f8b69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 00:48:25 +0200 Subject: [PATCH 352/496] Fix race condition in system.processes and Settings --- src/Backups/RestoreCoordinationRemote.cpp | 2 +- src/Backups/RestoreCoordinationRemote.h | 2 -- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 2 +- src/Interpreters/ProcessList.cpp | 2 +- 5 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 44214d00be5..0a69bc0eafb 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -323,7 +323,7 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic return false; bool result = false; - std::string path = zookeeper_path +"/stage"; + std::string path = zookeeper_path + "/stage"; auto holder = with_retries.createRetriesControlHolder("createRootNodes"); holder.retries_ctl.retryLoop( diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 9c299865cfa..a3d57e9a4d0 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -61,8 +61,6 @@ private: void createRootNodes(); void removeAllNodes(); - class ReplicatedDatabasesMetadataSync; - /// get_zookeeper will provide a zookeeper client without any fault injection const zkutil::GetZooKeeper get_zookeeper; const String root_zookeeper_path; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index ca30ee6db15..3fb6d30fcb8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -44,7 +44,7 @@ namespace ErrorCodes DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_) - : DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_) + : DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { } diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 4347649117d..41cfb751141 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -12,7 +12,7 @@ class DatabaseLazyIterator; class Context; /** Lazy engine of databases. - * Works like DatabaseOrdinary, but stores in memory only cache. + * Works like DatabaseOrdinary, but stores in memory only the cache. * Can be used only with *Log engines. */ class DatabaseLazy final : public DatabaseOnDisk diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 271e23a7288..6cb50b310ad 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -657,7 +657,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even { if (auto ctx = context.lock()) { - res.query_settings = std::make_shared(ctx->getSettingsRef()); + res.query_settings = std::make_shared(ctx->getSettingsCopy()); res.current_database = ctx->getCurrentDatabase(); } } From 572831f865d66e046d3e507d214e0f5aeae49ad4 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 2 Aug 2024 00:09:26 +0000 Subject: [PATCH 353/496] async_insert_race_long flakiness fixes 1. Make the test truly asynchronous. The setting `--async_insert_max_data_size 1` leads to data being flushed synchronously for all inserts in this test. This triggers part creation and extra resource consumption. 2. Do not run the `--wait_for_async_insert` query as a background process with a fixed (50ms) sleep time. If the actual execution time is longer than the anticipated delay time, it may lead to excessive process creation. --- tests/queries/0_stateless/02481_async_insert_race_long.sh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_race_long.sh b/tests/queries/0_stateless/02481_async_insert_race_long.sh index b0088017d32..91e6c4960e0 100755 --- a/tests/queries/0_stateless/02481_async_insert_race_long.sh +++ b/tests/queries/0_stateless/02481_async_insert_race_long.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -export MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert_busy_timeout_ms 10 --async_insert_max_data_size 1 --async_insert 1" +export MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert_busy_timeout_min_ms 50 --async_insert_busy_timeout_max_ms 50 --async_insert 1" function insert1() { @@ -29,11 +29,8 @@ function insert3() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" & - sleep 0.05 + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" done - - wait } function select1() From 0772ed7f6ac92bb2e016e5db00e85deafeecb127 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Fri, 2 Aug 2024 04:02:43 +0000 Subject: [PATCH 354/496] Code style feedback incorporated --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 10 +++++----- ...3209_parameterized_view_with_non_literal_params.sql | 2 ++ 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 6113a38d463..bffdba2f58a 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4549,11 +4549,11 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// Serialize the constant value using datatype specific /// interfaces to match the deserialization in ReplaceQueryParametersVistor. WriteBufferFromOwnString buf; - auto constval = constant->getValue(); - auto realtype = constant->getResultType(); - auto tempcol = realtype->createColumn(); - tempcol->insert(constval); - realtype->getDefaultSerialization()->serializeTextEscaped(*tempcol, 0, buf, {}); + const auto & value = constant->getValue(); + auto real_type = constant->getResultType(); + auto temporary_column = real_type->createColumn(); + temporary_column->insert(value); + real_type->getDefaultSerialization()->serializeTextEscaped(*temporary_column, 0, buf, {}); view_params[identifier_node->getIdentifier().getFullName()] = buf.str(); } } diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql index 55795c7a785..f2c61e5cb1d 100644 --- a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -90,8 +90,10 @@ select id from ipv4_pv(ipv4param=(select ipaddr from ipv4_table_pv where id=3)); drop view date_pv; drop view date_pv2; +drop view date32_pv; drop view uuid_pv; drop view ipv4_pv; drop table date_table_pv; +drop table date32_table_pv; drop table uuid_table_pv; drop table ipv4_table_pv; From d6da86dad282e6ad176b115d4344944daa8b9756 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 26 Jun 2024 01:27:47 +0000 Subject: [PATCH 355/496] Store plain_rewritable metadata in a separate layout --- .../CommonPathPrefixKeyGenerator.cpp | 6 +- .../MetadataStorageFromPlainObjectStorage.cpp | 36 ++++-- .../MetadataStorageFromPlainObjectStorage.h | 16 ++- ...torageFromPlainObjectStorageOperations.cpp | 71 +++++++---- ...aStorageFromPlainObjectStorageOperations.h | 14 ++- ...torageFromPlainRewritableObjectStorage.cpp | 119 +++++++++++++++--- ...aStorageFromPlainRewritableObjectStorage.h | 15 ++- .../test_s3_plain_rewritable/test.py | 13 ++ 8 files changed, 225 insertions(+), 65 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index e321c8a3c5a..2a06d56e5c7 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -19,11 +19,11 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); - auto key = std::filesystem::path(object_key_prefix.empty() ? storage_key_prefix : object_key_prefix); + auto key = std::filesystem::path(object_key_prefix.empty() ? std::string() : object_key_prefix); /// The longest prefix is the same as path, meaning that the path is already mapped. if (suffix_parts.empty()) - return ObjectStorageKey::createAsRelative(std::move(key)); + return ObjectStorageKey::createAsRelative(storage_key_prefix, std::move(key)); /// File and top-level directory paths are mapped as is. if (!is_directory || object_key_prefix.empty()) @@ -39,7 +39,7 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo key /= getRandomASCIIString(part_size); } - return ObjectStorageKey::createAsRelative(key); + return ObjectStorageKey::createAsRelative(storage_key_prefix, key); } std::tuple> CommonPathPrefixKeyGenerator::getLongestObjectKeyPrefix(const std::string & path) const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 30111d04d20..3da190c7256 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -7,6 +7,7 @@ #include #include +#include namespace DB { @@ -79,14 +80,16 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co object_storage->listObjects(abs_key, files, 0); - return getDirectChildrenOnDisk(abs_key, files, path); + std::unordered_set directories; + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const { /// Required for MergeTree auto paths = listDirectory(path); - // Prepend path, since iterateDirectory() includes path, unlike listDirectory() + /// Prepend path, since iterateDirectory() includes path, unlike listDirectory() std::for_each(paths.begin(), paths.end(), [&](auto & child) { child = fs::path(path) / child; }); std::vector fs_paths(paths.begin(), paths.end()); return std::make_unique(std::move(fs_paths)); @@ -99,10 +102,13 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std return {StoredObject(object_key.serialize(), path, object_size)}; } -std::vector MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & /* local_path */) const +void MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & /* storage_key_perfix */, + const RelativePathsWithMetadata & remote_paths, + const std::string & /* local_path */, + std::unordered_set & result) const { - std::unordered_set duplicates_filter; for (const auto & elem : remote_paths) { const auto & path = elem->relative_path; @@ -111,11 +117,10 @@ std::vector MetadataStorageFromPlainObjectStorage::getDirectChildre /// string::npos is ok. const auto slash_pos = path.find('/', child_pos); if (slash_pos == std::string::npos) - duplicates_filter.emplace(path.substr(child_pos)); + result.emplace(path.substr(child_pos)); else - duplicates_filter.emplace(path.substr(child_pos, slash_pos - child_pos)); + result.emplace(path.substr(child_pos, slash_pos - child_pos)); } - return std::vector(std::make_move_iterator(duplicates_filter.begin()), std::make_move_iterator(duplicates_filter.end())); } const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getStorageForNonTransactionalReads() const @@ -140,7 +145,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std else { addOperation(std::make_unique( - normalizeDirectoryPath(path), *metadata_storage.getPathMap(), object_storage)); + normalizeDirectoryPath(path), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix())); } } @@ -151,8 +156,13 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std auto normalized_path = normalizeDirectoryPath(path); auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path).serialize(); + chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto op = std::make_unique( - std::move(normalized_path), std::move(key_prefix), *metadata_storage.getPathMap(), object_storage); + std::move(normalized_path), + key_prefix.substr(object_storage->getCommonKeyPrefix().size()), + *metadata_storage.getPathMap(), + object_storage, + metadata_storage.getMetadataKeyPrefix()); addOperation(std::move(op)); } @@ -167,7 +177,11 @@ void MetadataStorageFromPlainObjectStorageTransaction::moveDirectory(const std:: throwNotImplemented(); addOperation(std::make_unique( - normalizeDirectoryPath(path_from), normalizeDirectoryPath(path_to), *metadata_storage.getPathMap(), object_storage)); + normalizeDirectoryPath(path_from), + normalizeDirectoryPath(path_to), + *metadata_storage.getPathMap(), + object_storage, + metadata_storage.getMetadataKeyPrefix())); } void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 66da0f2431e..97c5715a937 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace DB { @@ -78,10 +80,20 @@ public: bool supportsStat() const override { return false; } protected: + /// Get the object storage prefix for storing metadata files. If stored behind a separate endpoint, + /// the metadata keys reflect the layout of the regular files. + virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } + + /// Returns a map of local paths to paths in object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } - virtual std::vector getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const; + /// Retrieves the immediate files and directories within a given directory on a disk. + virtual void getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const; }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 7e4b1f69962..0a6086bd39d 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -20,14 +20,24 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; +ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const std::string & metadata_key_prefix) +{ + auto prefix = std::filesystem::path(metadata_key_prefix) / key_prefix; + return ObjectStorageKey::createAsRelative(prefix.string(), PREFIX_PATH_FILE_NAME); +} } MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_) - : path(std::move(path_)), key_prefix(key_prefix_), path_map(path_map_), object_storage(object_storage_) + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path(std::move(path_)) + , key_prefix(key_prefix_) + , path_map(path_map_) + , object_storage(object_storage_) + , metadata_key_prefix(metadata_key_prefix_) { } @@ -36,13 +46,17 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: if (path_map.contains(path)) return; - LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Creating metadata for directory '{}'", path); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); + LOG_TRACE( + getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), + "Creating metadata for directory '{}' with remote path='{}'", + path, + metadata_object_key.serialize()); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, @@ -66,25 +80,31 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + if (write_finalized) { path_map.erase(path); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); - object_storage->removeObject(StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); + object_storage->removeObject(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } else if (write_created) - object_storage->removeObjectIfExists(StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); + object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_) - : path_from(std::move(path_from_)), path_to(std::move(path_to_)), path_map(path_map_), object_storage(object_storage_) + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path_from(std::move(path_from_)) + , path_to(std::move(path_to_)) + , path_map(path_map_) + , object_storage(object_storage_) + , metadata_key_prefix(metadata_key_prefix_) { } @@ -98,26 +118,26 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo if (path_map.contains(new_path)) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); - auto object_key = ObjectStorageKey::createAsRelative(expected_it->second, PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); - auto object = StoredObject(object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); if (validate_content) { std::string data; - auto read_buf = object_storage->readObject(object); + auto read_buf = object_storage->readObject(metadata_object); readStringUntilEOF(data, *read_buf); if (data != path_from) throw Exception( ErrorCodes::INCORRECT_DATA, "Incorrect data for object key {}, expected {}, got {}", - object_key.serialize(), + metadata_object_key.serialize(), expected_path, data); } auto write_buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /*buf_size*/ DBMS_DEFAULT_BUFFER_SIZE, @@ -156,8 +176,11 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq } MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, ObjectStoragePtr object_storage_) - : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_) + std::filesystem::path && path_, + MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } @@ -170,9 +193,9 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); key_prefix = path_it->second; - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); - object_storage->removeObject(object); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + object_storage->removeObject(metadata_object); path_map.erase(path_it); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -189,10 +212,10 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un if (!removed) return; - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 4b196f787fd..e31e3cbb262 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -16,6 +16,7 @@ private: std::string key_prefix; MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; bool write_created = false; bool write_finalized = false; @@ -26,7 +27,8 @@ public: std::filesystem::path && path_, std::string && key_prefix_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_); + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; void undo(std::unique_lock & metadata_lock) override; @@ -39,6 +41,7 @@ private: std::filesystem::path path_to; MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; bool write_created = false; bool write_finalized = false; @@ -51,7 +54,8 @@ public: std::filesystem::path && path_from_, std::filesystem::path && path_to_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_); + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; @@ -65,13 +69,17 @@ private: MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; std::string key_prefix; bool removed = false; public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, ObjectStoragePtr object_storage_); + std::filesystem::path && path_, + MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; void undo(std::unique_lock & metadata_lock) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 7718fba9c28..f3d00a928e3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -21,8 +22,22 @@ namespace { constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; +constexpr auto METADATA_PATH_TOKEN = "__meta/"; -MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & root, ObjectStoragePtr object_storage) +/// Use a separate layout for metadata iff: +/// 1. The disk endpoint does not contain objects, OR +/// 2. The metadata is already stored behind a separate endpoint. +/// Otherwise, store metadata along with regular data for backward compatibility. +std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) +{ + const auto common_key_prefix = std::filesystem::path(object_storage->getCommonKeyPrefix()); + const auto metadata_key_prefix = std::filesystem::path(common_key_prefix) / METADATA_PATH_TOKEN; + return !object_storage->existsOrHasAnyChild(metadata_key_prefix / "") && object_storage->existsOrHasAnyChild(common_key_prefix / "") + ? common_key_prefix + : metadata_key_prefix; +} + +MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { MetadataStorageFromPlainObjectStorage::PathMap result; @@ -39,16 +54,16 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri LOG_DEBUG(log, "Loading metadata"); size_t num_files = 0; - for (auto iterator = object_storage->iterate(root, 0); iterator->isValid(); iterator->next()) + for (auto iterator = object_storage->iterate(metadata_key_prefix, 0); iterator->isValid(); iterator->next()) { ++num_files; auto file = iterator->current(); String path = file->getPath(); - auto remote_path = std::filesystem::path(path); - if (remote_path.filename() != PREFIX_PATH_FILE_NAME) + auto remote_metadata_path = std::filesystem::path(path); + if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_path, path, &object_storage, &result, &mutex, &log, &settings] + runner([remote_metadata_path, path, &object_storage, &result, &mutex, &log, &settings, &metadata_key_prefix] { setThreadName("PlainRWMetaLoad"); @@ -75,7 +90,10 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri throw; } - chassert(remote_path.has_parent_path()); + chassert(remote_metadata_path.has_parent_path()); + chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); + auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); + auto remote_path = std::filesystem::path(std::move(suffix)); std::pair res; { std::lock_guard lock(mutex); @@ -103,17 +121,17 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri return result; } -std::vector getDirectChildrenOnRewritableDisk( +void getDirectChildrenOnRewritableDisk( const std::string & storage_key, + const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, const MetadataStorageFromPlainObjectStorage::PathMap & local_path_prefixes, - SharedMutex & shared_mutex) + SharedMutex & shared_mutex, + std::unordered_set & result) { using PathMap = MetadataStorageFromPlainObjectStorage::PathMap; - std::unordered_set duplicates_filter; - /// Map remote paths into local subdirectories. std::unordered_map remote_to_local_subdir; @@ -149,22 +167,21 @@ std::vector getDirectChildrenOnRewritableDisk( /// File names. auto filename = path.substr(child_pos); if (!skip_list.contains(filename)) - duplicates_filter.emplace(std::move(filename)); + result.emplace(std::move(filename)); } else { /// Subdirectories. - auto it = remote_to_local_subdir.find(path.substr(0, slash_pos)); + chassert(path.find(storage_key_perfix) == 0); + auto it = remote_to_local_subdir.find(path.substr(storage_key_perfix.size(), slash_pos - storage_key_perfix.size())); /// Mapped subdirectories. if (it != remote_to_local_subdir.end()) - duplicates_filter.emplace(it->second); + result.emplace(it->second); /// The remote subdirectory name is the same as the local subdirectory. else - duplicates_filter.emplace(path.substr(child_pos, slash_pos - child_pos)); + result.emplace(path.substr(child_pos, slash_pos - child_pos)); } } - - return std::vector(std::make_move_iterator(duplicates_filter.begin()), std::make_move_iterator(duplicates_filter.end())); } } @@ -172,7 +189,8 @@ std::vector getDirectChildrenOnRewritableDisk( MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewritableObjectStorage( ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) - , path_map(std::make_shared(loadPathPrefixMap(object_storage->getCommonKeyPrefix(), object_storage))) + , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) + , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) { if (object_storage->isWriteOnce()) throw Exception( @@ -190,10 +208,71 @@ MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewrit CurrentMetrics::sub(metric, path_map->size()); } -std::vector MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const +bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & path) const { - return getDirectChildrenOnRewritableDisk(storage_key, remote_paths, local_path, *getPathMap(), metadata_mutex); + if (MetadataStorageFromPlainObjectStorage::exists(path)) + return true; + + if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + { + auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); + return object_storage->existsOrHasAnyChild(metadata_key); + } + + return false; +} + +bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::string & path) const +{ + if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + { + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path).serialize()) / ""; + chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key + = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); + return object_storage->existsOrHasAnyChild(metadata_key); + } + else + return MetadataStorageFromPlainObjectStorage::isDirectory(path); +} + +std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const +{ + auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + + RelativePathsWithMetadata files; + std::string abs_key = key_prefix; + if (!abs_key.ends_with('/')) + abs_key += '/'; + + object_storage->listObjects(abs_key, files, 0); + + std::unordered_set directories; + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove + /// metadata along with regular files. + if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) + { + chassert(abs_key.starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / abs_key.substr(object_storage->getCommonKeyPrefix().size()); + RelativePathsWithMetadata metadata_files; + object_storage->listObjects(metadata_key, metadata_files, 0); + getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); + } + + return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); +} + +void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const +{ + getDirectChildrenOnRewritableDisk(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index a5394b9428d..71153cbdc25 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -11,6 +12,7 @@ namespace DB class MetadataStorageFromPlainRewritableObjectStorage final : public MetadataStorageFromPlainObjectStorage { private: + const std::string metadata_key_prefix; std::shared_ptr path_map; public: @@ -18,11 +20,20 @@ public: ~MetadataStorageFromPlainRewritableObjectStorage() override; MetadataStorageType getType() const override { return MetadataStorageType::PlainRewritable; } + bool exists(const std::string & path) const override; + bool isDirectory(const std::string & path) const override; + std::vector listDirectory(const std::string & path) const override; + protected: + std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } - std::vector getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const override; + void getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const override; }; } diff --git a/tests/integration/test_s3_plain_rewritable/test.py b/tests/integration/test_s3_plain_rewritable/test.py index 4b1aaafc814..020e170eb48 100644 --- a/tests/integration/test_s3_plain_rewritable/test.py +++ b/tests/integration/test_s3_plain_rewritable/test.py @@ -139,6 +139,19 @@ def test(storage_policy): == insert_values_arr[i] ) + metadata_it = cluster.minio_client.list_objects( + cluster.minio_bucket, "data/", recursive=True + ) + metadata_count = 0 + for obj in list(metadata_it): + if "/__meta/" in obj.object_name: + assert obj.object_name.endswith("/prefix.path") + metadata_count += 1 + else: + assert not obj.object_name.endswith("/prefix.path") + + assert metadata_count > 0 + for i in range(NUM_WORKERS): node = cluster.instances[f"node{i + 1}"] node.query("DROP TABLE IF EXISTS test SYNC") From 98ad45ba960de4cc29ee794e2eeccf9fe6f8e0a8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 00:58:20 +0000 Subject: [PATCH 356/496] Add prefix_path parameter to object key generator --- src/Common/ObjectStorageKeyGenerator.cpp | 11 ++++++++--- src/Common/ObjectStorageKeyGenerator.h | 3 ++- .../AzureBlobStorage/AzureObjectStorage.cpp | 4 +++- .../AzureBlobStorage/AzureObjectStorage.h | 2 +- .../Cached/CachedObjectStorage.cpp | 10 ++++++---- .../ObjectStorages/Cached/CachedObjectStorage.h | 5 +++-- .../CommonPathPrefixKeyGenerator.cpp | 3 ++- .../CommonPathPrefixKeyGenerator.h | 3 ++- .../DiskObjectStorageTransaction.cpp | 6 +++--- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 ++++-- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 5 +++-- .../ObjectStorages/Local/LocalObjectStorage.cpp | 14 ++++++++------ .../ObjectStorages/Local/LocalObjectStorage.h | 2 +- .../MetadataStorageFromPlainObjectStorage.cpp | 14 +++++++------- ...aStorageFromPlainRewritableObjectStorage.cpp | 6 +++--- src/Disks/ObjectStorages/PlainObjectStorage.h | 2 +- .../PlainRewritableObjectStorage.h | 17 +++++++++++------ src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 4 ++-- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 +- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 2 +- 22 files changed, 74 insertions(+), 51 deletions(-) diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index e9212c3f04d..3e7bf3116bd 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -14,7 +15,10 @@ public: , re_gen(key_template) { } - DB::ObjectStorageKey generate(const String &, bool) const override { return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); } + DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional & /* key_prefix */) const override + { + return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); + } private: String key_template; @@ -29,7 +33,7 @@ public: : key_prefix(std::move(key_prefix_)) {} - DB::ObjectStorageKey generate(const String &, bool) const override + DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional & /* key_prefix */) const override { /// Path to store the new S3 object. @@ -60,7 +64,8 @@ public: : key_prefix(std::move(key_prefix_)) {} - DB::ObjectStorageKey generate(const String & path, bool) const override + DB::ObjectStorageKey + generate(const String & path, bool /* is_directory */, const std::optional & /* key_prefix */) const override { return DB::ObjectStorageKey::createAsRelative(key_prefix, path); } diff --git a/src/Common/ObjectStorageKeyGenerator.h b/src/Common/ObjectStorageKeyGenerator.h index 11da039b33b..12aeec1714d 100644 --- a/src/Common/ObjectStorageKeyGenerator.h +++ b/src/Common/ObjectStorageKeyGenerator.h @@ -11,7 +11,8 @@ class IObjectStorageKeysGenerator public: virtual ~IObjectStorageKeysGenerator() = default; - virtual ObjectStorageKey generate(const String & path, bool is_directory) const = 0; + /// Generates an object storage key based on a path in the virtual filesystem. + virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const = 0; }; using ObjectStorageKeysGeneratorPtr = std::shared_ptr; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index bc16955143b..0d92561d142 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -1,3 +1,4 @@ +#include #include #include "Common/Exception.h" @@ -117,7 +118,8 @@ AzureObjectStorage::AzureObjectStorage( { } -ObjectStorageKey AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { return ObjectStorageKey::createAsRelative(getRandomASCIIString(32)); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 2c7ce5e18dc..bc90b05e64d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -101,7 +101,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return true; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index a3b6e25e8ea..fb817005399 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -34,14 +34,16 @@ FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const return cache->createKeyForPath(path); } -ObjectStorageKey CachedObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey +CachedObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { - return object_storage->generateObjectKeyForPath(path); + return object_storage->generateObjectKeyForPath(path, key_prefix); } -ObjectStorageKey CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const +ObjectStorageKey +CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const { - return object_storage->generateObjectKeyPrefixForDirectoryPath(path); + return object_storage->generateObjectKeyPrefixForDirectoryPath(path, key_prefix); } ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settings) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 93ef2659cbb..efcdbfebabf 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -98,9 +98,10 @@ public: const std::string & getCacheName() const override { return cache_config_name; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; - ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; + ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const override; void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { object_storage->setKeysGenerator(gen); } diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 2a06d56e5c7..0a4426e8e66 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -15,7 +15,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( { } -ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory) const +ObjectStorageKey +CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & /* key_prefix */) const { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index fb1140de908..08495738505 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -26,7 +27,7 @@ public: explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); - ObjectStorageKey generate(const String & path, bool is_directory) const override; + ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; private: /// Longest key prefix and unresolved parts of the source path. diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index b5805f6d23a..880911b9958 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -537,7 +537,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation for (const auto & object_from : source_blobs) { - auto object_key = destination_object_storage.generateObjectKeyForPath(to_path); + auto object_key = destination_object_storage.generateObjectKeyForPath(to_path, std::nullopt /* key_prefix */); auto object_to = StoredObject(object_key.serialize()); object_storage.copyObjectToAnotherObjectStorage(object_from, object_to,read_settings,write_settings, destination_object_storage); @@ -738,7 +738,7 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile const WriteSettings & settings, bool autocommit) { - auto object_key = object_storage.generateObjectKeyForPath(path); + auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; if (metadata_helper) @@ -835,7 +835,7 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) { /// This function is a simplified and adapted version of DiskObjectStorageTransaction::writeFile(). - auto object_key = object_storage.generateObjectKeyForPath(path); + auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; if (metadata_helper) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index dcb2af9d4d3..3ce2a0f4903 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -4,8 +4,9 @@ #include #include -#include +#include #include +#include #include #include @@ -53,7 +54,8 @@ std::string HDFSObjectStorage::extractObjectKeyFromURL(const StoredObject & obje return path; } -ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { initializeHDFSFS(); /// what ever data_source_description.description value is, consider that key as relative key diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 8aae90d0721..0cb31eb8b8b 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -111,7 +111,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return true; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index ceea4d5a2bb..529c79790fd 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -232,10 +232,11 @@ public: /// Generate blob name for passed absolute local path. /// Path can be generated either independently or based on `path`. - virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path) const = 0; + virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const = 0; /// Object key prefix for local paths in the directory 'path'. - virtual ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */) const + virtual ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'generateObjectKeyPrefixForDirectoryPath' is not implemented"); } diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index a247d86ddce..20ef135cdf7 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,15 +1,16 @@ #include -#include -#include -#include +#include +#include +#include #include #include -#include #include #include +#include +#include #include -#include +#include namespace fs = std::filesystem; @@ -222,7 +223,8 @@ std::unique_ptr LocalObjectStorage::cloneObjectStorage( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage() is not implemented for LocalObjectStorage"); } -ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { constexpr size_t key_name_total_size = 32; return ObjectStorageKey::createAsRelative(key_prefix, getRandomASCIIString(key_name_total_size)); diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 371cd37f8b2..564d49bf876 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -81,7 +81,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return false; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 3da190c7256..589b18abca8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -42,7 +42,7 @@ bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) con { /// NOTE: exists() cannot be used here since it works only for existing /// key, and does not work for some intermediate path. - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); return object_storage->existsOrHasAnyChild(object_key.serialize()); } @@ -54,7 +54,7 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); auto directory = std::filesystem::path(std::move(key_prefix)) / ""; return object_storage->existsOrHasAnyChild(directory); @@ -62,7 +62,7 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); auto metadata = object_storage->tryGetObjectMetadata(object_key.serialize()); if (metadata) return metadata->size_bytes; @@ -71,7 +71,7 @@ uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); RelativePathsWithMetadata files; std::string abs_key = key_prefix; @@ -98,7 +98,7 @@ DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(con StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std::string & path) const { size_t object_size = getFileSize(path); - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); return {StoredObject(object_key.serialize(), path, object_size)}; } @@ -130,7 +130,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { - auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path); + auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); auto object = StoredObject(object_key.serialize()); metadata_storage.object_storage->removeObject(object); } @@ -155,7 +155,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path).serialize(); + auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, std::nullopt /* key_prefix */).serialize(); chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto op = std::make_unique( std::move(normalized_path), diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index f3d00a928e3..de65cd5c233 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -215,7 +215,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); return object_storage->existsOrHasAnyChild(metadata_key); @@ -228,7 +228,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str { if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path).serialize()) / ""; + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize()) / ""; chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); @@ -240,7 +240,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); RelativePathsWithMetadata files; std::string abs_key = key_prefix; diff --git a/src/Disks/ObjectStorages/PlainObjectStorage.h b/src/Disks/ObjectStorages/PlainObjectStorage.h index e0907d0b4d8..805b3436fce 100644 --- a/src/Disks/ObjectStorages/PlainObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainObjectStorage.h @@ -26,7 +26,7 @@ public: bool isPlain() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & /* key_prefix */) const override { return ObjectStorageKey::createAsRelative(BaseObjectStorage::getCommonKeyPrefix(), path); } diff --git a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h index 5f000afe625..dcea5964fc5 100644 --- a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include #include "CommonPathPrefixKeyGenerator.h" @@ -33,9 +35,10 @@ public: bool isPlain() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; - ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; + ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const override; void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; } @@ -46,20 +49,22 @@ private: template -ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyForPath( + const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ false); + return key_generator->generate(path, /* is_directory */ false, key_prefix); } template -ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyPrefixForDirectoryPath( + const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ true); + return key_generator->generate(path, /* is_directory */ true, key_prefix); } } diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index 63e7ebb00c5..b20a2940e47 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -79,7 +79,7 @@ bool checkBatchRemove(S3ObjectStorage & storage) /// We are using generateObjectKeyForPath() which returns random object key. /// That generated key is placed in a right directory where we should have write access. const String path = fmt::format("clickhouse_remove_objects_capability_{}", getServerUUID()); - const auto key = storage.generateObjectKeyForPath(path); + const auto key = storage.generateObjectKeyForPath(path, {} /* key_prefix */); StoredObject object(key.serialize(), path); try { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a6672e14e10..3c4b4d76bf5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -624,12 +624,12 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( std::move(new_client), std::move(new_s3_settings), new_uri, s3_capabilities, key_generator, disk_name); } -ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ false); + return key_generator->generate(path, /* is_directory */ false, key_prefix); } std::shared_ptr S3ObjectStorage::getS3StorageClient() diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index cbe004bc298..d786a6b37f3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -164,7 +164,7 @@ public: bool supportParallelWrite() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isReadOnly() const override { return s3_settings.get()->read_only; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 9ca2950dae0..ab357d6f50d 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -82,7 +82,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & /* key_prefix */) const override { return ObjectStorageKey::createAsRelative(path); } From 27392fee6eec22c7f2dac3d17f73ab9a528f1fc8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 01:30:49 +0000 Subject: [PATCH 357/496] Minor refactor --- .../CommonPathPrefixKeyGenerator.cpp | 8 ++-- .../MetadataStorageFromPlainObjectStorage.cpp | 38 +++++++------------ .../MetadataStorageFromPlainObjectStorage.h | 8 ---- ...torageFromPlainRewritableObjectStorage.cpp | 26 +++++-------- ...aStorageFromPlainRewritableObjectStorage.h | 2 +- 5 files changed, 28 insertions(+), 54 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 0a4426e8e66..ef599a2f366 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -16,15 +16,15 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( } ObjectStorageKey -CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & /* key_prefix */) const +CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); - auto key = std::filesystem::path(object_key_prefix.empty() ? std::string() : object_key_prefix); + auto key = std::filesystem::path(object_key_prefix); /// The longest prefix is the same as path, meaning that the path is already mapped. if (suffix_parts.empty()) - return ObjectStorageKey::createAsRelative(storage_key_prefix, std::move(key)); + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, std::move(key)); /// File and top-level directory paths are mapped as is. if (!is_directory || object_key_prefix.empty()) @@ -40,7 +40,7 @@ CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, c key /= getRandomASCIIString(part_size); } - return ObjectStorageKey::createAsRelative(storage_key_prefix, key); + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, key); } std::tuple> CommonPathPrefixKeyGenerator::getLongestObjectKeyPrefix(const std::string & path) const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 589b18abca8..02048c07a57 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -80,9 +80,20 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co object_storage->listObjects(abs_key, files, 0); - std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); - return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); + std::unordered_set result; + for (const auto & elem : files) + { + const auto & p = elem->relative_path; + chassert(p.find(abs_key) == 0); + const auto child_pos = abs_key.size(); + /// string::npos is ok. + const auto slash_pos = p.find('/', child_pos); + if (slash_pos == std::string::npos) + result.emplace(p.substr(child_pos)); + else + result.emplace(p.substr(child_pos, slash_pos - child_pos)); + } + return std::vector(std::make_move_iterator(result.begin()), std::make_move_iterator(result.end())); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const @@ -102,27 +113,6 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std return {StoredObject(object_key.serialize(), path, object_size)}; } -void MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, - const std::string & /* storage_key_perfix */, - const RelativePathsWithMetadata & remote_paths, - const std::string & /* local_path */, - std::unordered_set & result) const -{ - for (const auto & elem : remote_paths) - { - const auto & path = elem->relative_path; - chassert(path.find(storage_key) == 0); - const auto child_pos = storage_key.size(); - /// string::npos is ok. - const auto slash_pos = path.find('/', child_pos); - if (slash_pos == std::string::npos) - result.emplace(path.substr(child_pos)); - else - result.emplace(path.substr(child_pos, slash_pos - child_pos)); - } -} - const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getStorageForNonTransactionalReads() const { return metadata_storage; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 97c5715a937..237327cd1f4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -86,14 +86,6 @@ protected: /// Returns a map of local paths to paths in object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } - - /// Retrieves the immediate files and directories within a given directory on a disk. - virtual void getDirectChildrenOnDisk( - const std::string & storage_key, - const std::string & storage_key_perfix, - const RelativePathsWithMetadata & remote_paths, - const std::string & local_path, - std::unordered_set & result) const; }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index de65cd5c233..b904c0d92b9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -121,7 +121,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri return result; } -void getDirectChildrenOnRewritableDisk( +void getDirectChildrenOnDiskImpl( const std::string & storage_key, const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, @@ -215,10 +215,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); - chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); - return object_storage->existsOrHasAnyChild(metadata_key); + auto key_prefix = object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize(); + return object_storage->existsOrHasAnyChild(key_prefix); } return false; @@ -228,11 +226,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str { if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize()) / ""; - chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key - = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); - return object_storage->existsOrHasAnyChild(metadata_key); + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize()) / ""; + return object_storage->existsOrHasAnyChild(directory); } else return MetadataStorageFromPlainObjectStorage::isDirectory(path); @@ -240,12 +235,10 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, "" /* key_prefix */).serialize(); RelativePathsWithMetadata files; - std::string abs_key = key_prefix; - if (!abs_key.ends_with('/')) - abs_key += '/'; + auto abs_key = std::filesystem::path(object_storage->getCommonKeyPrefix()) / key_prefix / ""; object_storage->listObjects(abs_key, files, 0); @@ -255,8 +248,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) { - chassert(abs_key.starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / abs_key.substr(object_storage->getCommonKeyPrefix().size()); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; object_storage->listObjects(metadata_key, metadata_files, 0); getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); @@ -272,7 +264,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnRewritableDisk(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); + getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index 71153cbdc25..b067b391878 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -33,7 +33,7 @@ protected: const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - std::unordered_set & result) const override; + std::unordered_set & result) const; }; } From ecca720f9e076e49f280e48c6ff4046a19894b2a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 06:42:38 +0000 Subject: [PATCH 358/496] minor --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 02048c07a57..7553c7733b5 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -145,11 +145,10 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, std::nullopt /* key_prefix */).serialize(); - chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); + auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, "" /* key_prefix */).serialize(); auto op = std::make_unique( std::move(normalized_path), - key_prefix.substr(object_storage->getCommonKeyPrefix().size()), + std::move(key_prefix), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix()); From 97519ae800b9a26942973d888354a2b013d53cc6 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 4 Jul 2024 07:02:13 +0000 Subject: [PATCH 359/496] in-memory map path comparator --- .../CommonPathPrefixKeyGenerator.cpp | 5 +++-- .../CommonPathPrefixKeyGenerator.h | 4 +++- .../MetadataStorageFromPlainObjectStorage.h | 3 ++- ...torageFromPlainObjectStorageOperations.cpp | 18 ++++++++--------- ...torageFromPlainRewritableObjectStorage.cpp | 9 ++++----- src/Disks/ObjectStorages/PathComparator.h | 20 +++++++++++++++++++ 6 files changed, 41 insertions(+), 18 deletions(-) create mode 100644 src/Disks/ObjectStorages/PathComparator.h diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index ef599a2f366..062a2542654 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -18,7 +18,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { - const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); + const auto & [object_key_prefix, suffix_parts] + = getLongestObjectKeyPrefix(is_directory ? std::filesystem::path(path).parent_path().string() : path); auto key = std::filesystem::path(object_key_prefix); @@ -54,7 +55,7 @@ std::tuple> CommonPathPrefixKeyGenerator:: while (p != p.root_path()) { - auto it = ptr->find(p / ""); + auto it = ptr->find(p); if (it != ptr->end()) { std::vector vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end())); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index 08495738505..bca4f7060c4 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include @@ -23,7 +25,7 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: /// Local to remote path map. Leverages filesystem::path comparator for paths. - using PathMap = std::map; + using PathMap = std::map; explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 237327cd1f4..9ea1c475821 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -29,7 +30,7 @@ class MetadataStorageFromPlainObjectStorage : public IMetadataStorage { public: /// Local path prefixes mapped to storage key prefixes. - using PathMap = std::map; + using PathMap = std::map; private: friend class MetadataStorageFromPlainObjectStorageTransaction; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 0a6086bd39d..b0b384f62c7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -43,7 +43,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - if (path_map.contains(path)) + if (path_map.contains(path.parent_path())) return; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -64,7 +64,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; - [[maybe_unused]] auto result = path_map.emplace(path, std::move(key_prefix)); + [[maybe_unused]] auto result = path_map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); @@ -84,7 +84,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un if (write_finalized) { - path_map.erase(path); + path_map.erase(path.parent_path()); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -111,11 +111,11 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto expected_it = path_map.find(expected_path); + auto expected_it = path_map.find(expected_path.parent_path()); if (expected_it == path_map.end()) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); - if (path_map.contains(new_path)) + if (path_map.contains(new_path.parent_path())) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); @@ -156,7 +156,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - [[maybe_unused]] auto result = path_map.emplace(path_to, path_map.extract(path_from).mapped()); + [[maybe_unused]] auto result = path_map.emplace(path_to.parent_path(), path_map.extract(path_from.parent_path()).mapped()); chassert(result.second); write_finalized = true; @@ -165,7 +165,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock &) { if (write_finalized) - path_map.emplace(path_from, path_map.extract(path_to).mapped()); + path_map.emplace(path_from.parent_path(), path_map.extract(path_to.parent_path()).mapped()); if (write_created) { @@ -186,7 +186,7 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto path_it = path_map.find(path); + auto path_it = path_map.find(path.parent_path()); if (path_it == path_map.end()) return; @@ -223,7 +223,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - path_map.emplace(path, std::move(key_prefix)); + path_map.emplace(path.parent_path(), std::move(key_prefix)); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index b904c0d92b9..ba8dfc891dd 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -97,7 +97,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri std::pair res; { std::lock_guard lock(mutex); - res = result.emplace(local_path, remote_path.parent_path()); + res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); } /// This can happen if table replication is enabled, then the same local path is written @@ -145,11 +145,10 @@ void getDirectChildrenOnDiskImpl( break; auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); - if (slash_num != 1) + if (slash_num != 0) continue; - chassert(k.back() == '/'); - remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end() - 1)); + remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/"); } } @@ -243,7 +242,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi object_storage->listObjects(abs_key, files, 0); std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/PathComparator.h new file mode 100644 index 00000000000..fe97a465937 --- /dev/null +++ b/src/Disks/ObjectStorages/PathComparator.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ +// TODO: rename +struct PathComparator +{ + bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + { + auto d1 = std::distance(path1.begin(), path1.end()); + auto d2 = std::distance(path2.begin(), path2.end()); + if (d1 != d2) + return d1 < d2; + return path1 < path2; + } +}; + +} From aa290b6398a5affa8405d3584795bce6bf7450d4 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 10 Jul 2024 01:15:57 +0000 Subject: [PATCH 360/496] use a designated mutex for path_map --- .../CommonPathPrefixKeyGenerator.cpp | 13 ++- .../CommonPathPrefixKeyGenerator.h | 6 +- .../MetadataStorageFromPlainObjectStorage.h | 6 +- ...torageFromPlainObjectStorageOperations.cpp | 99 ++++++++++++++----- ...aStorageFromPlainObjectStorageOperations.h | 13 +-- ...torageFromPlainRewritableObjectStorage.cpp | 25 ++--- ...aStorageFromPlainRewritableObjectStorage.h | 4 +- src/Disks/ObjectStorages/PathComparator.h | 27 +++-- 8 files changed, 122 insertions(+), 71 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 062a2542654..19dd819fc17 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,4 +1,5 @@ #include "CommonPathPrefixKeyGenerator.h" +#include "Disks/ObjectStorages/PathComparator.h" #include @@ -9,9 +10,8 @@ namespace DB { -CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( - String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_) - : storage_key_prefix(key_prefix_), shared_mutex(shared_mutex_), path_map(std::move(path_map_)) +CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator(String key_prefix_, std::weak_ptr path_map_) + : storage_key_prefix(key_prefix_), path_map(std::move(path_map_)) { } @@ -49,14 +49,13 @@ std::tuple> CommonPathPrefixKeyGenerator:: std::filesystem::path p(path); std::deque dq; - std::shared_lock lock(shared_mutex); - auto ptr = path_map.lock(); + std::shared_lock lock(ptr->mutex); while (p != p.root_path()) { - auto it = ptr->find(p); - if (it != ptr->end()) + auto it = ptr->map.find(p); + if (it != ptr->map.end()) { std::vector vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end())); return std::make_tuple(it->second, std::move(vec)); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index bca4f7060c4..e337745b627 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -25,9 +25,8 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: /// Local to remote path map. Leverages filesystem::path comparator for paths. - using PathMap = std::map; - explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); + explicit CommonPathPrefixKeyGenerator(String key_prefix_, std::weak_ptr path_map_); ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; @@ -37,8 +36,7 @@ private: const String storage_key_prefix; - SharedMutex & shared_mutex; - std::weak_ptr path_map; + std::weak_ptr path_map; }; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 9ea1c475821..dfb9632666c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -28,10 +28,6 @@ using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; - private: friend class MetadataStorageFromPlainObjectStorageTransaction; @@ -86,7 +82,7 @@ protected: virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } /// Returns a map of local paths to paths in object storage. - virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } + virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index b0b384f62c7..b4a85efbaab 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,4 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" +#include "Disks/ObjectStorages/PathComparator.h" #include #include @@ -30,7 +31,7 @@ ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const s MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)) @@ -43,8 +44,13 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - if (path_map.contains(path.parent_path())) - return; + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::shared_lock lock(mutex); + if (map.contains(path.parent_path())) + return; + } auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -64,8 +70,11 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; - [[maybe_unused]] auto result = path_map.emplace(path.parent_path(), std::move(key_prefix)); - chassert(result.second); + { + std::unique_lock lock(mutex); + [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); + chassert(result.second); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); @@ -80,11 +89,17 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { + auto & map = path_map.map; + auto & mutex = path_map.mutex; + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); if (write_finalized) { - path_map.erase(path.parent_path()); + { + std::unique_lock lock(mutex); + map.erase(path.parent_path()); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -97,7 +112,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path_from(std::move(path_from_)) @@ -111,14 +126,25 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto expected_it = path_map.find(expected_path.parent_path()); - if (expected_it == path_map.end()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); + auto & map = path_map.map; + auto & mutex = path_map.mutex; - if (path_map.contains(new_path.parent_path())) - throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); + std::filesystem::path remote_path; + { + std::shared_lock lock(mutex); + auto expected_it = map.find(expected_path.parent_path()); + if (expected_it == map.end()) + throw Exception( + ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); - auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); + if (map.contains(new_path.parent_path())) + throw Exception( + ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); + + remote_path = expected_it->second; + } + + auto metadata_object_key = createMetadataObjectKey(remote_path, metadata_key_prefix); auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); @@ -156,8 +182,13 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - [[maybe_unused]] auto result = path_map.emplace(path_to.parent_path(), path_map.extract(path_from.parent_path()).mapped()); - chassert(result.second); + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::unique_lock lock(mutex); + [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); + chassert(result.second); + } write_finalized = true; } @@ -165,7 +196,12 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock &) { if (write_finalized) - path_map.emplace(path_from.parent_path(), path_map.extract(path_to.parent_path()).mapped()); + { + auto & map = path_map.map; + auto & mutex = path_map.mutex; + std::unique_lock lock(mutex); + map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); + } if (write_created) { @@ -176,28 +212,34 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq } MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_, - const std::string & metadata_key_prefix_) + std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto path_it = path_map.find(path.parent_path()); - if (path_it == path_map.end()) - return; + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::shared_lock lock(mutex); + auto path_it = map.find(path.parent_path()); + if (path_it == map.end()) + return; + key_prefix = path_it->second; + } LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); - key_prefix = path_it->second; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); object_storage->removeObject(metadata_object); - path_map.erase(path_it); + { + std::unique_lock lock(mutex); + map.erase(path.parent_path()); + } + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -223,7 +265,12 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - path_map.emplace(path.parent_path(), std::move(key_prefix)); + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::unique_lock lock(mutex); + map.emplace(path.parent_path(), std::move(key_prefix)); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index e31e3cbb262..1b2471dd316 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -2,6 +2,7 @@ #include #include +#include "Disks/ObjectStorages/PathComparator.h" #include #include @@ -14,7 +15,7 @@ class MetadataStorageFromPlainObjectStorageCreateDirectoryOperation final : publ private: std::filesystem::path path; std::string key_prefix; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -26,7 +27,7 @@ public: MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); @@ -39,7 +40,7 @@ class MetadataStorageFromPlainObjectStorageMoveDirectoryOperation final : public private: std::filesystem::path path_from; std::filesystem::path path_to; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -53,7 +54,7 @@ public: MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); @@ -67,7 +68,7 @@ class MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation final : publ private: std::filesystem::path path; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -77,7 +78,7 @@ private: public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index ba8dfc891dd..cf51a6a5314 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -3,11 +3,12 @@ #include #include -#include #include +#include #include #include #include "CommonPathPrefixKeyGenerator.h" +#include "Disks/ObjectStorages/PathComparator.h" namespace DB @@ -37,9 +38,10 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) +InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { - MetadataStorageFromPlainObjectStorage::PathMap result; + using Map = InMemoryPathMap::Map; + Map result; ThreadPool & pool = getIOThreadPool().get(); ThreadPoolCallbackRunnerLocal runner(pool, "PlainRWMetaLoad"); @@ -94,7 +96,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); auto remote_path = std::filesystem::path(std::move(suffix)); - std::pair res; + std::pair res; { std::lock_guard lock(mutex); res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); @@ -126,14 +128,13 @@ void getDirectChildrenOnDiskImpl( const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - const MetadataStorageFromPlainObjectStorage::PathMap & local_path_prefixes, + const InMemoryPathMap::Map & local_path_prefixes, SharedMutex & shared_mutex, std::unordered_set & result) { - using PathMap = MetadataStorageFromPlainObjectStorage::PathMap; - /// Map remote paths into local subdirectories. - std::unordered_map remote_to_local_subdir; + using Map = InMemoryPathMap::Map; + std::unordered_map remote_to_local_subdir; { std::shared_lock lock(shared_mutex); @@ -189,7 +190,7 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) - , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) + , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) { if (object_storage->isWriteOnce()) throw Exception( @@ -197,14 +198,14 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), metadata_mutex, path_map); + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage() { auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, path_map->size()); + CurrentMetrics::sub(metric, path_map->map.size()); } bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & path) const @@ -263,7 +264,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); + getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index b067b391878..fea461abab8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -13,7 +13,7 @@ class MetadataStorageFromPlainRewritableObjectStorage final : public MetadataSto { private: const std::string metadata_key_prefix; - std::shared_ptr path_map; + std::shared_ptr path_map; public: MetadataStorageFromPlainRewritableObjectStorage(ObjectStoragePtr object_storage_, String storage_path_prefix_); @@ -27,7 +27,7 @@ public: protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } - std::shared_ptr getPathMap() const override { return path_map; } + std::shared_ptr getPathMap() const override { return path_map; } void getDirectChildrenOnDisk( const std::string & storage_key, const std::string & storage_key_perfix, diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/PathComparator.h index fe97a465937..fae82108789 100644 --- a/src/Disks/ObjectStorages/PathComparator.h +++ b/src/Disks/ObjectStorages/PathComparator.h @@ -1,20 +1,29 @@ #pragma once #include +#include +#include "Common/SharedMutex.h" namespace DB { -// TODO: rename -struct PathComparator + + +struct InMemoryPathMap { - bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + struct PathComparator { - auto d1 = std::distance(path1.begin(), path1.end()); - auto d2 = std::distance(path2.begin(), path2.end()); - if (d1 != d2) - return d1 < d2; - return path1 < path2; - } + bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + { + auto d1 = std::distance(path1.begin(), path1.end()); + auto d2 = std::distance(path2.begin(), path2.end()); + if (d1 != d2) + return d1 < d2; + return path1 < path2; + } + }; + using Map = std::map; + Map map; + SharedMutex mutex; }; } From 0e78ed6b580646cc08721eef415ffb3fe2f697cb Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 10 Jul 2024 23:04:21 +0000 Subject: [PATCH 361/496] simplify listDirectory --- ...torageFromPlainRewritableObjectStorage.cpp | 28 ++++--------------- ...aStorageFromPlainRewritableObjectStorage.h | 1 - 2 files changed, 5 insertions(+), 24 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index cf51a6a5314..6a0eff0a136 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -125,17 +125,12 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, void getDirectChildrenOnDiskImpl( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, const InMemoryPathMap::Map & local_path_prefixes, SharedMutex & shared_mutex, std::unordered_set & result) { - /// Map remote paths into local subdirectories. - using Map = InMemoryPathMap::Map; - std::unordered_map remote_to_local_subdir; - { std::shared_lock lock(shared_mutex); auto end_it = local_path_prefixes.end(); @@ -147,9 +142,9 @@ void getDirectChildrenOnDiskImpl( auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); if (slash_num != 0) - continue; + break; - remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/"); + result.emplace(std::string(k.begin() + local_path.size(), k.end()) + "/"); } } @@ -169,18 +164,6 @@ void getDirectChildrenOnDiskImpl( if (!skip_list.contains(filename)) result.emplace(std::move(filename)); } - else - { - /// Subdirectories. - chassert(path.find(storage_key_perfix) == 0); - auto it = remote_to_local_subdir.find(path.substr(storage_key_perfix.size(), slash_pos - storage_key_perfix.size())); - /// Mapped subdirectories. - if (it != remote_to_local_subdir.end()) - result.emplace(it->second); - /// The remote subdirectory name is the same as the local subdirectory. - else - result.emplace(path.substr(child_pos, slash_pos - child_pos)); - } } } @@ -243,7 +226,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi object_storage->listObjects(abs_key, files, 0); std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, std::filesystem::path(path) / "", directories); + getDirectChildrenOnDisk(abs_key, files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) @@ -251,7 +234,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; object_storage->listObjects(metadata_key, metadata_files, 0); - getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); + getDirectChildrenOnDisk(metadata_key, metadata_files, std::filesystem::path(path) / "", directories); } return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); @@ -259,12 +242,11 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); + getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index fea461abab8..8fd147e15b9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -30,7 +30,6 @@ protected: std::shared_ptr getPathMap() const override { return path_map; } void getDirectChildrenOnDisk( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const; From 82f5aceb484a322960065f973bec2b61c31219aa Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 11 Jul 2024 06:22:26 +0000 Subject: [PATCH 362/496] introduce flat structure --- .../FlatStructureKeyGenerator.cpp | 51 +++++++++++++++++++ .../FlatStructureKeyGenerator.h | 23 +++++++++ ...torageFromPlainRewritableObjectStorage.cpp | 15 ++++-- 3 files changed, 86 insertions(+), 3 deletions(-) create mode 100644 src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp create mode 100644 src/Disks/ObjectStorages/FlatStructureKeyGenerator.h diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp new file mode 100644 index 00000000000..d6fb32b65d4 --- /dev/null +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp @@ -0,0 +1,51 @@ +#include "FlatStructureKeyGenerator.h" +#include "Common/ObjectStorageKey.h" +#include "Common/SharedMutex.h" +#include "Disks/ObjectStorages/PathComparator.h" +#include + +#include +#include +#include + +namespace DB +{ + +FlatStructureKeyGenerator::FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) + : storage_key_prefix(storage_key_prefix_), path_map(std::move(path_map_)) +{ +} + +ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const +{ + if (is_directory) + chassert(path.ends_with('/')); + + const auto p = std::filesystem::path(path); + auto directory = p.parent_path(); + + constexpr size_t part_size = 32; + + std::optional remote_path; + { + auto ptr = path_map.lock(); + std::shared_lock lock(ptr->mutex); + auto it = ptr->map.find(p); + if (it != ptr->map.end()) + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, it->second); + + it = ptr->map.find(directory); + if (it != ptr->map.end()) + remote_path = it->second; + } + std::filesystem::path key = remote_path.has_value() ? *remote_path + : is_directory ? std::filesystem::path(getRandomASCIIString(part_size)) + : directory; + + if (!is_directory) + key /= p.filename(); + + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, key); +} + +} diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h new file mode 100644 index 00000000000..2c585dffb81 --- /dev/null +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +#include +namespace DB +{ + +class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator +{ +public: + explicit FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); + + ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; + +private: + const String storage_key_prefix; + + std::weak_ptr path_map; +}; + +} diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 6a0eff0a136..afaa7bf06ff 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include #include @@ -8,7 +10,6 @@ #include #include #include "CommonPathPrefixKeyGenerator.h" -#include "Disks/ObjectStorages/PathComparator.h" namespace DB @@ -181,8 +182,16 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); - object_storage->setKeysGenerator(keys_gen); + if (getMetadataKeyPrefix() == object_storage->getCommonKeyPrefix()) + { + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + object_storage->setKeysGenerator(keys_gen); + } + else + { + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + object_storage->setKeysGenerator(keys_gen); + } } MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage() From c0e6780dfe5977316e50e587877f3fe6ef11d048 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 11 Jul 2024 22:48:07 +0000 Subject: [PATCH 363/496] rename PathComparator.h -> InMemoryPathMap.h --- src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp | 4 ++-- src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h | 5 ++--- src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp | 2 +- src/Disks/ObjectStorages/FlatStructureKeyGenerator.h | 2 +- .../ObjectStorages/{PathComparator.h => InMemoryPathMap.h} | 0 .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 1 + .../ObjectStorages/MetadataStorageFromPlainObjectStorage.h | 3 ++- .../MetadataStorageFromPlainObjectStorageOperations.cpp | 2 +- .../MetadataStorageFromPlainObjectStorageOperations.h | 2 +- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 4 ++-- 10 files changed, 13 insertions(+), 12 deletions(-) rename src/Disks/ObjectStorages/{PathComparator.h => InMemoryPathMap.h} (100%) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 19dd819fc17..1fa06823bae 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,5 +1,5 @@ -#include "CommonPathPrefixKeyGenerator.h" -#include "Disks/ObjectStorages/PathComparator.h" +#include +#include #include diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index e337745b627..8b5037e3804 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -1,9 +1,6 @@ #pragma once #include -#include - -#include #include #include @@ -21,6 +18,8 @@ namespace DB /// /// The key generator ensures that the original directory hierarchy is /// preserved, which is required for the MergeTree family. + +struct InMemoryPathMap; class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp index d6fb32b65d4..414aea2b08b 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp @@ -1,7 +1,7 @@ #include "FlatStructureKeyGenerator.h" +#include #include "Common/ObjectStorageKey.h" #include "Common/SharedMutex.h" -#include "Disks/ObjectStorages/PathComparator.h" #include #include diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h index 2c585dffb81..6b5b2203bed 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h @@ -1,12 +1,12 @@ #pragma once -#include #include #include namespace DB { +struct InMemoryPathMap; class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator { public: diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/InMemoryPathMap.h similarity index 100% rename from src/Disks/ObjectStorages/PathComparator.h rename to src/Disks/ObjectStorages/InMemoryPathMap.h diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 7553c7733b5..364d04e2b52 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -1,5 +1,6 @@ #include "MetadataStorageFromPlainObjectStorage.h" #include +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index dfb9632666c..a9a1a648f96 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -2,9 +2,9 @@ #include #include +#include #include #include -#include #include #include @@ -13,6 +13,7 @@ namespace DB { +struct InMemoryPathMap; struct UnlinkMetadataFileOperationOutcome; using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index b4a85efbaab..6f5109faec4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,5 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" -#include "Disks/ObjectStorages/PathComparator.h" +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 1b2471dd316..778585fa758 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -1,8 +1,8 @@ #pragma once #include +#include #include -#include "Disks/ObjectStorages/PathComparator.h" #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index afaa7bf06ff..c312eae4077 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,7 +1,7 @@ -#include #include +#include +#include #include -#include #include #include From 3f066018fb0c74783ee486f54fc472ffd9cd7cc1 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 12 Jul 2024 02:27:45 +0000 Subject: [PATCH 364/496] style and doc --- src/Common/ObjectStorageKeyGenerator.cpp | 1 - src/Common/ObjectStorageKeyGenerator.h | 4 ++++ .../CommonPathPrefixKeyGenerator.h | 1 + ...=> FlatDirectoryStructureKeyGenerator.cpp} | 9 ++++--- ...h => FlatDirectoryStructureKeyGenerator.h} | 4 ++-- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 - src/Disks/ObjectStorages/InMemoryPathMap.h | 2 +- .../Local/LocalObjectStorage.cpp | 1 - .../MetadataStorageFromPlainObjectStorage.cpp | 2 -- .../MetadataStorageFromPlainObjectStorage.h | 5 ++-- ...torageFromPlainObjectStorageOperations.cpp | 24 +++++++++---------- ...aStorageFromPlainObjectStorageOperations.h | 3 +-- ...torageFromPlainRewritableObjectStorage.cpp | 12 +++++++--- 13 files changed, 35 insertions(+), 34 deletions(-) rename src/Disks/ObjectStorages/{FlatStructureKeyGenerator.cpp => FlatDirectoryStructureKeyGenerator.cpp} (80%) rename src/Disks/ObjectStorages/{FlatStructureKeyGenerator.h => FlatDirectoryStructureKeyGenerator.h} (64%) diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index 3e7bf3116bd..3bdc0004198 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -3,7 +3,6 @@ #include #include -#include #include diff --git a/src/Common/ObjectStorageKeyGenerator.h b/src/Common/ObjectStorageKeyGenerator.h index 12aeec1714d..008e3c88fac 100644 --- a/src/Common/ObjectStorageKeyGenerator.h +++ b/src/Common/ObjectStorageKeyGenerator.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "ObjectStorageKey.h" namespace DB @@ -12,6 +13,9 @@ public: virtual ~IObjectStorageKeysGenerator() = default; /// Generates an object storage key based on a path in the virtual filesystem. + /// @param path - Path in the virtual filesystem. + /// @param is_directory - If the path in the virtual filesystem corresponds to a directory. + /// @param key_prefix - Optional key prefix for the generated object storage key. If provided, this prefix will be added to the beginning of the generated key. virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const = 0; }; diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index 8b5037e3804..ea91d78600d 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -9,6 +9,7 @@ namespace DB { +/// Deprecated. Used for backward compatibility with plain rewritable disks without a separate metadata layout. /// Object storage key generator used specifically with the /// MetadataStorageFromPlainObjectStorage if multiple writes are allowed. diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp similarity index 80% rename from src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp rename to src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp index 414aea2b08b..64959b729b6 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp @@ -1,4 +1,4 @@ -#include "FlatStructureKeyGenerator.h" +#include "FlatDirectoryStructureKeyGenerator.h" #include #include "Common/ObjectStorageKey.h" #include "Common/SharedMutex.h" @@ -11,12 +11,12 @@ namespace DB { -FlatStructureKeyGenerator::FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) +FlatDirectoryStructureKeyGenerator::FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) : storage_key_prefix(storage_key_prefix_), path_map(std::move(path_map_)) { } -ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const +ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { if (is_directory) chassert(path.ends_with('/')); @@ -24,8 +24,6 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i const auto p = std::filesystem::path(path); auto directory = p.parent_path(); - constexpr size_t part_size = 32; - std::optional remote_path; { auto ptr = path_map.lock(); @@ -38,6 +36,7 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i if (it != ptr->map.end()) remote_path = it->second; } + constexpr size_t part_size = 32; std::filesystem::path key = remote_path.has_value() ? *remote_path : is_directory ? std::filesystem::path(getRandomASCIIString(part_size)) : directory; diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h similarity index 64% rename from src/Disks/ObjectStorages/FlatStructureKeyGenerator.h rename to src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h index 6b5b2203bed..4dbac5d3003 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h @@ -7,10 +7,10 @@ namespace DB { struct InMemoryPathMap; -class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator +class FlatDirectoryStructureKeyGenerator : public IObjectStorageKeysGenerator { public: - explicit FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); + explicit FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 3ce2a0f4903..00ef4b63e6f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index fae82108789..ea08784719e 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -2,7 +2,7 @@ #include #include -#include "Common/SharedMutex.h" +#include namespace DB { diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 20ef135cdf7..5b61c57ca21 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 364d04e2b52..2036208c389 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -146,10 +146,8 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, "" /* key_prefix */).serialize(); auto op = std::make_unique( std::move(normalized_path), - std::move(key_prefix), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix()); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index a9a1a648f96..2aac7158bd5 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -78,11 +78,10 @@ public: bool supportsStat() const override { return false; } protected: - /// Get the object storage prefix for storing metadata files. If stored behind a separate endpoint, - /// the metadata keys reflect the layout of the regular files. + /// Get the object storage prefix for storing metadata files. virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } - /// Returns a map of local paths to paths in object storage. + /// Returns a map of virtual filesystem paths to paths in the object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 6f5109faec4..9e18f6cdb08 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -29,25 +29,21 @@ ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const s } MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( - std::filesystem::path && path_, - std::string && key_prefix_, - InMemoryPathMap & path_map_, - ObjectStoragePtr object_storage_, - const std::string & metadata_key_prefix_) + std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)) - , key_prefix(key_prefix_) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) + , key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* key_prefix */).serialize()) { } void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::shared_lock lock(mutex); + auto & map = path_map.map; if (map.contains(path.parent_path())) return; } @@ -72,6 +68,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::unique_lock lock(mutex); + auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); } @@ -89,7 +86,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto & map = path_map.map; auto & mutex = path_map.mutex; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -98,6 +94,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un { { std::unique_lock lock(mutex); + auto & map = path_map.map; map.erase(path.parent_path()); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -126,12 +123,12 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto & map = path_map.map; auto & mutex = path_map.mutex; std::filesystem::path remote_path; { std::shared_lock lock(mutex); + auto & map = path_map.map; auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) throw Exception( @@ -182,10 +179,10 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::unique_lock lock(mutex); + auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); chassert(result.second); } @@ -197,9 +194,9 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq { if (write_finalized) { - auto & map = path_map.map; auto & mutex = path_map.mutex; std::unique_lock lock(mutex); + auto & map = path_map.map; map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); } @@ -219,10 +216,10 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::shared_lock lock(mutex); + auto & map = path_map.map; auto path_it = map.find(path.parent_path()); if (path_it == map.end()) return; @@ -237,6 +234,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: { std::unique_lock lock(mutex); + auto & map = path_map.map; map.erase(path.parent_path()); } @@ -265,10 +263,10 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::unique_lock lock(mutex); + auto & map = path_map.map; map.emplace(path.parent_path(), std::move(key_prefix)); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 778585fa758..3ac0ffef8d2 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -14,10 +14,10 @@ class MetadataStorageFromPlainObjectStorageCreateDirectoryOperation final : publ { private: std::filesystem::path path; - std::string key_prefix; InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; + const std::string key_prefix; bool write_created = false; bool write_finalized = false; @@ -26,7 +26,6 @@ public: // Assuming that paths are normalized. MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, - std::string && key_prefix_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index c312eae4077..fd3b9523df6 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -132,16 +132,20 @@ void getDirectChildrenOnDiskImpl( SharedMutex & shared_mutex, std::unordered_set & result) { + /// Directories are retrieved from the in-memory path map. { std::shared_lock lock(shared_mutex); auto end_it = local_path_prefixes.end(); for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) { - const auto & [k, v] = std::make_tuple(it->first.string(), it->second); + const auto & [k, _] = std::make_tuple(it->first.string(), it->second); if (!k.starts_with(local_path)) break; auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); + /// The local_path_prefixes comparator ensures that the paths with the smallest number of + /// hops from the local_path are iterated first. The paths do not end with '/', hence + /// break the loop if the number of slashes is greater than 0. if (slash_num != 0) break; @@ -149,6 +153,7 @@ void getDirectChildrenOnDiskImpl( } } + /// Files. auto skip_list = std::set{PREFIX_PATH_FILE_NAME}; for (const auto & elem : remote_paths) { @@ -189,7 +194,8 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita } else { - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + /// Use flat directory structure if the metadata is stored separately from the table data. + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } } From d4c13714abb6b307c4344c74bd4b7973c03e68df Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 13 Jul 2024 06:51:02 +0000 Subject: [PATCH 365/496] address feedback: TSA_GUARDED_BY --- .../CommonPathPrefixKeyGenerator.cpp | 5 +-- .../FlatDirectoryStructureKeyGenerator.cpp | 7 ++-- src/Disks/ObjectStorages/InMemoryPathMap.h | 4 +-- ...torageFromPlainObjectStorageOperations.cpp | 34 +++++++------------ ...torageFromPlainRewritableObjectStorage.cpp | 34 +++++++++++-------- 5 files changed, 40 insertions(+), 44 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 1fa06823bae..1d041626a7e 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -49,8 +50,8 @@ std::tuple> CommonPathPrefixKeyGenerator:: std::filesystem::path p(path); std::deque dq; - auto ptr = path_map.lock(); - std::shared_lock lock(ptr->mutex); + const auto ptr = path_map.lock(); + SharedLockGuard lock(ptr->mutex); while (p != p.root_path()) { diff --git a/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp index 64959b729b6..0f35bfd2427 100644 --- a/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp @@ -1,7 +1,8 @@ #include "FlatDirectoryStructureKeyGenerator.h" #include #include "Common/ObjectStorageKey.h" -#include "Common/SharedMutex.h" +#include +#include #include #include @@ -26,8 +27,8 @@ ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & pat std::optional remote_path; { - auto ptr = path_map.lock(); - std::shared_lock lock(ptr->mutex); + const auto ptr = path_map.lock(); + SharedLockGuard lock(ptr->mutex); auto it = ptr->map.find(p); if (it != ptr->map.end()) return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, it->second); diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index ea08784719e..dcd28dfaf6c 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -22,8 +22,8 @@ struct InMemoryPathMap } }; using Map = std::map; - Map map; - SharedMutex mutex; + mutable SharedMutex mutex; + Map map TSA_GUARDED_BY(mutex); }; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 9e18f6cdb08..8a06b204cfc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,4 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" +#include "Common/SharedLockGuard.h" #include #include @@ -40,11 +41,9 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - auto & mutex = path_map.mutex; { - std::shared_lock lock(mutex); - auto & map = path_map.map; - if (map.contains(path.parent_path())) + SharedLockGuard lock(path_map.mutex); + if (path_map.map.contains(path.parent_path())) return; } @@ -67,7 +66,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); @@ -86,16 +85,13 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto & mutex = path_map.mutex; - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); if (write_finalized) { { - std::unique_lock lock(mutex); - auto & map = path_map.map; - map.erase(path.parent_path()); + std::lock_guard lock(path_map.mutex); + path_map.map.erase(path.parent_path()); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -123,11 +119,9 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto & mutex = path_map.mutex; - std::filesystem::path remote_path; { - std::shared_lock lock(mutex); + SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) @@ -179,9 +173,8 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - auto & mutex = path_map.mutex; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); chassert(result.second); @@ -194,8 +187,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq { if (write_finalized) { - auto & mutex = path_map.mutex; - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); } @@ -216,9 +208,8 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto & mutex = path_map.mutex; { - std::shared_lock lock(mutex); + SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; auto path_it = map.find(path.parent_path()); if (path_it == map.end()) @@ -233,7 +224,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: object_storage->removeObject(metadata_object); { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.erase(path.parent_path()); } @@ -263,9 +254,8 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - auto & mutex = path_map.mutex; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.emplace(path.parent_path(), std::move(key_prefix)); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index fd3b9523df6..22e73e36372 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -9,6 +9,8 @@ #include #include #include +#include "Common/SharedLockGuard.h" +#include "Common/SharedMutex.h" #include "CommonPathPrefixKeyGenerator.h" @@ -39,14 +41,13 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) +std::shared_ptr loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { + auto result = std::make_shared(); using Map = InMemoryPathMap::Map; - Map result; ThreadPool & pool = getIOThreadPool().get(); ThreadPoolCallbackRunnerLocal runner(pool, "PlainRWMetaLoad"); - std::mutex mutex; LoggerPtr log = getLogger("MetadataStorageFromPlainObjectStorage"); @@ -66,7 +67,7 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_metadata_path, path, &object_storage, &result, &mutex, &log, &settings, &metadata_key_prefix] + runner([remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] { setThreadName("PlainRWMetaLoad"); @@ -99,8 +100,8 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, auto remote_path = std::filesystem::path(std::move(suffix)); std::pair res; { - std::lock_guard lock(mutex); - res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); + std::lock_guard lock(result->mutex); + res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); } /// This can happen if table replication is enabled, then the same local path is written @@ -117,10 +118,13 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, } runner.waitForAllToFinishAndRethrowFirstError(); - LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result.size()); + { + SharedLockGuard lock(result->mutex); + LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size()); - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::add(metric, result.size()); + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::add(metric, result->map.size()); + } return result; } @@ -128,14 +132,14 @@ void getDirectChildrenOnDiskImpl( const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - const InMemoryPathMap::Map & local_path_prefixes, - SharedMutex & shared_mutex, + const InMemoryPathMap & path_map, std::unordered_set & result) { /// Directories are retrieved from the in-memory path map. { - std::shared_lock lock(shared_mutex); - auto end_it = local_path_prefixes.end(); + SharedLockGuard lock(path_map.mutex); + const auto & local_path_prefixes = path_map.map; + const auto end_it = local_path_prefixes.end(); for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) { const auto & [k, _] = std::make_tuple(it->first.string(), it->second); @@ -179,7 +183,7 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) - , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) + , path_map(loadPathPrefixMap(metadata_key_prefix, object_storage)) { if (object_storage->isWriteOnce()) throw Exception( @@ -261,7 +265,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); + getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result); } } From db13ba2c488303e90717fbcc5adf5304241ac474 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 13 Jul 2024 06:51:40 +0000 Subject: [PATCH 366/496] style fix --- .../CommonPathPrefixKeyGenerator.cpp | 2 +- ...torageFromPlainObjectStorageOperations.cpp | 2 +- ...torageFromPlainRewritableObjectStorage.cpp | 91 ++++++++++--------- 3 files changed, 48 insertions(+), 47 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 1d041626a7e..521d5c037ab 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,8 +1,8 @@ #include #include -#include #include +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 8a06b204cfc..76090411bb9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,6 +1,6 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" -#include "Common/SharedLockGuard.h" #include +#include "Common/SharedLockGuard.h" #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 22e73e36372..dba63bba321 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -7,10 +7,10 @@ #include #include #include -#include -#include #include "Common/SharedLockGuard.h" #include "Common/SharedMutex.h" +#include +#include #include "CommonPathPrefixKeyGenerator.h" @@ -67,54 +67,55 @@ std::shared_ptr loadPathPrefixMap(const std::string & metadata_ if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] - { - setThreadName("PlainRWMetaLoad"); - - StoredObject object{path}; - String local_path; - - try + runner( + [remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] { - auto read_buf = object_storage->readObject(object, settings); - readStringUntilEOF(local_path, *read_buf); - } + setThreadName("PlainRWMetaLoad"); + + StoredObject object{path}; + String local_path; + + try + { + auto read_buf = object_storage->readObject(object, settings); + readStringUntilEOF(local_path, *read_buf); + } #if USE_AWS_S3 - catch (const S3Exception & e) - { - /// It is ok if a directory was removed just now. - /// We support attaching a filesystem that is concurrently modified by someone else. - if (e.getS3ErrorCode() == Aws::S3::S3Errors::NO_SUCH_KEY) - return; - throw; - } + catch (const S3Exception & e) + { + /// It is ok if a directory was removed just now. + /// We support attaching a filesystem that is concurrently modified by someone else. + if (e.getS3ErrorCode() == Aws::S3::S3Errors::NO_SUCH_KEY) + return; + throw; + } #endif - catch (...) - { - throw; - } + catch (...) + { + throw; + } - chassert(remote_metadata_path.has_parent_path()); - chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); - auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); - auto remote_path = std::filesystem::path(std::move(suffix)); - std::pair res; - { - std::lock_guard lock(result->mutex); - res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); - } + chassert(remote_metadata_path.has_parent_path()); + chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); + auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); + auto remote_path = std::filesystem::path(std::move(suffix)); + std::pair res; + { + std::lock_guard lock(result->mutex); + res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); + } - /// This can happen if table replication is enabled, then the same local path is written - /// in `prefix.path` of each replica. - /// TODO: should replicated tables (e.g., RMT) be explicitly disallowed? - if (!res.second) - LOG_WARNING( - log, - "The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'", - local_path, - res.first->second, - remote_path.parent_path().string()); - }); + /// This can happen if table replication is enabled, then the same local path is written + /// in `prefix.path` of each replica. + /// TODO: should replicated tables (e.g., RMT) be explicitly disallowed? + if (!res.second) + LOG_WARNING( + log, + "The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'", + local_path, + res.first->second, + remote_path.parent_path().string()); + }); } runner.waitForAllToFinishAndRethrowFirstError(); From 912bddf86f53f207b76ba453e43b6724b24ef6df Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:49:37 -0700 Subject: [PATCH 367/496] Update src/Disks/ObjectStorages/InMemoryPathMap.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/ObjectStorages/InMemoryPathMap.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index dcd28dfaf6c..2ac291dfaf0 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -21,6 +21,7 @@ struct InMemoryPathMap return path1 < path2; } }; + /// Local -> Remote path. using Map = std::map; mutable SharedMutex mutex; Map map TSA_GUARDED_BY(mutex); From 727f5ed108e3b92c81d4ed295e0de438de8bae2b Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:49:57 -0700 Subject: [PATCH 368/496] Update src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index dba63bba321..3380dec60ca 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -29,7 +29,7 @@ constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; constexpr auto METADATA_PATH_TOKEN = "__meta/"; /// Use a separate layout for metadata iff: -/// 1. The disk endpoint does not contain objects, OR +/// 1. The disk endpoint does not contain any objects yet (empty), OR /// 2. The metadata is already stored behind a separate endpoint. /// Otherwise, store metadata along with regular data for backward compatibility. std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) From 4c78531c9c0681a84309e02ecfde17a36f1c1ad5 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:50:20 -0700 Subject: [PATCH 369/496] Update src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 3380dec60ca..40aed32c047 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -28,7 +28,7 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; constexpr auto METADATA_PATH_TOKEN = "__meta/"; -/// Use a separate layout for metadata iff: +/// Use a separate layout for metadata if: /// 1. The disk endpoint does not contain any objects yet (empty), OR /// 2. The metadata is already stored behind a separate endpoint. /// Otherwise, store metadata along with regular data for backward compatibility. From 3b986ef3400021cf18797a7872b926dcc191b547 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 05:50:42 +0000 Subject: [PATCH 370/496] address feedback: useSeparateLayoutForMetadata --- ...StorageFromPlainRewritableObjectStorage.cpp | 18 +++++++++++------- ...taStorageFromPlainRewritableObjectStorage.h | 4 +++- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 40aed32c047..39b11d9a3e3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -192,15 +192,15 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - if (getMetadataKeyPrefix() == object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + /// Use flat directory structure if the metadata is stored separately from the table data. + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } else { - /// Use flat directory structure if the metadata is stored separately from the table data. - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } } @@ -216,7 +216,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (MetadataStorageFromPlainObjectStorage::exists(path)) return true; - if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto key_prefix = object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize(); return object_storage->existsOrHasAnyChild(key_prefix); @@ -227,7 +227,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::string & path) const { - if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize()) / ""; return object_storage->existsOrHasAnyChild(directory); @@ -249,7 +249,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi getDirectChildrenOnDisk(abs_key, files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. - if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; @@ -269,4 +269,8 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result); } +bool MetadataStorageFromPlainRewritableObjectStorage::useSeparateLayoutForMetadata() const +{ + return getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix(); +} } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index 8fd147e15b9..82d93e3e7ae 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -24,7 +24,6 @@ public: bool isDirectory(const std::string & path) const override; std::vector listDirectory(const std::string & path) const override; - protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } @@ -33,6 +32,9 @@ protected: const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const; + +private: + bool useSeparateLayoutForMetadata() const; }; } From 359b42738a25aa02436c1bebc49d0b751e456ccb Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 06:11:51 +0000 Subject: [PATCH 371/496] address feedback: key_prefix -> object_key_prefix --- ...tadataStorageFromPlainObjectStorageOperations.cpp | 12 ++++++------ ...MetadataStorageFromPlainObjectStorageOperations.h | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 76090411bb9..31fb8c7ef97 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -22,9 +22,9 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; -ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const std::string & metadata_key_prefix) +ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, const std::string & metadata_key_prefix) { - auto prefix = std::filesystem::path(metadata_key_prefix) / key_prefix; + auto prefix = std::filesystem::path(metadata_key_prefix) / object_key_prefix; return ObjectStorageKey::createAsRelative(prefix.string(), PREFIX_PATH_FILE_NAME); } } @@ -35,7 +35,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) - , key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* key_prefix */).serialize()) + , object_key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* object_key_prefix */).serialize()) { } @@ -47,7 +47,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: return; } - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); LOG_TRACE( getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), @@ -68,7 +68,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); + [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(object_key_prefix)); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -85,7 +85,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); if (write_finalized) { diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 3ac0ffef8d2..02305767faf 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -17,7 +17,7 @@ private: InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; - const std::string key_prefix; + const std::string object_key_prefix; bool write_created = false; bool write_finalized = false; From 79a8cbe0c595e877a750c26cc27a8c68202279c7 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 06:16:00 +0000 Subject: [PATCH 372/496] address feedback: documentation --- .../MetadataStorageFromPlainObjectStorageOperations.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 31fb8c7ef97..be5168c5385 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -55,7 +55,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: path, metadata_object_key.serialize()); - auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( metadata_object, WriteMode::Rewrite, @@ -137,7 +137,8 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo auto metadata_object_key = createMetadataObjectKey(remote_path, metadata_key_prefix); - auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); + auto metadata_object + = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ expected_path / PREFIX_PATH_FILE_NAME); if (validate_content) { @@ -220,7 +221,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); - auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME); object_storage->removeObject(metadata_object); { From 42bd49dae6244590ba406ad502083bf610276eb9 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 07:05:51 +0000 Subject: [PATCH 373/496] address feedback: parent_path() for directories --- ...torageFromPlainObjectStorageOperations.cpp | 30 ++++++++++++------- ...aStorageFromPlainObjectStorageOperations.h | 4 ++- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index be5168c5385..c0e3f8e1fc9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -31,7 +31,7 @@ ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path(std::move(path_)) + : path((chassert(path_.string().ends_with('/')), std::move(path_))) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) @@ -41,9 +41,11 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { + /// parent_path() removes the trailing '/' + const auto base_path = path.parent_path(); { SharedLockGuard lock(path_map.mutex); - if (path_map.map.contains(path.parent_path())) + if (path_map.map.contains(base_path)) return; } @@ -68,7 +70,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(object_key_prefix)); + [[maybe_unused]] auto result = map.emplace(base_path, std::move(object_key_prefix)); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -89,9 +91,10 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un if (write_finalized) { + const auto base_path = path.parent_path(); { std::lock_guard lock(path_map.mutex); - path_map.map.erase(path.parent_path()); + path_map.map.erase(base_path); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -108,8 +111,8 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path_from(std::move(path_from_)) - , path_to(std::move(path_to_)) + : path_from((chassert(path_from_.string().ends_with('/')), std::move(path_from_))) + , path_to((chassert(path_to_.string().ends_with('/')), std::move(path_to_))) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) @@ -123,6 +126,7 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo { SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; + /// parent_path() removes the trailing '/'. auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) throw Exception( @@ -174,10 +178,14 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); + /// parent_path() removes the trailing '/'. + auto base_path_to = path_to.parent_path(); + auto base_path_from = path_from.parent_path(); + { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); + [[maybe_unused]] auto result = map.emplace(base_path_to, map.extract(base_path_from).mapped()); chassert(result.second); } @@ -203,16 +211,18 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) + : path((chassert(path_.string().ends_with('/')), std::move(path_))), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { + /// parent_path() removes the trailing '/' + const auto base_path = path.parent_path(); { SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; - auto path_it = map.find(path.parent_path()); + auto path_it = map.find(base_path); if (path_it == map.end()) return; key_prefix = path_it->second; @@ -227,7 +237,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - map.erase(path.parent_path()); + map.erase(base_path); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 02305767faf..93ebe668d56 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -23,8 +23,8 @@ private: bool write_finalized = false; public: - // Assuming that paths are normalized. MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( + /// path_ must end with a trailing '/'. std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, @@ -51,6 +51,7 @@ private: public: MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( + /// Both path_from_ and path_to_ must end with a trailing '/'. std::filesystem::path && path_from_, std::filesystem::path && path_to_, InMemoryPathMap & path_map_, @@ -76,6 +77,7 @@ private: public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( + /// path_ must end with a trailing '/'. std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, From 41fc84bb2df441d117681c589dbea5c516ed4748 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 21:16:27 +0000 Subject: [PATCH 374/496] fix build --- src/Disks/ObjectStorages/InMemoryPathMap.h | 1 + ...taStorageFromPlainObjectStorageOperations.cpp | 16 ++++++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index 2ac291dfaf0..e319c187ca7 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index c0e3f8e1fc9..bfd203ef2e0 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,10 +1,10 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" #include -#include "Common/SharedLockGuard.h" #include #include #include +#include #include namespace DB @@ -31,12 +31,13 @@ ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path((chassert(path_.string().ends_with('/')), std::move(path_))) + : path(std::move(path_)) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) , object_key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* object_key_prefix */).serialize()) { + chassert(path.string().ends_with('/')); } void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) @@ -70,7 +71,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(base_path, std::move(object_key_prefix)); + [[maybe_unused]] auto result = map.emplace(base_path, object_key_prefix); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -111,12 +112,14 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path_from((chassert(path_from_.string().ends_with('/')), std::move(path_from_))) - , path_to((chassert(path_to_.string().ends_with('/')), std::move(path_to_))) + : path_from(std::move(path_from_)) + , path_to(std::move(path_to_)) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) { + chassert(path_from.string().ends_with('/')); + chassert(path_to.string().ends_with('/')); } std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( @@ -211,8 +214,9 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path((chassert(path_.string().ends_with('/')), std::move(path_))), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) + : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { + chassert(path.string().ends_with('/')); } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) From 9b4e02e8dabb649076389ee96a271da025913ddf Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 16 Jul 2024 22:48:37 +0000 Subject: [PATCH 375/496] fix macOs build --- src/Disks/ObjectStorages/InMemoryPathMap.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index e319c187ca7..a9859d5e2b8 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -25,7 +25,13 @@ struct InMemoryPathMap /// Local -> Remote path. using Map = std::map; mutable SharedMutex mutex; - Map map TSA_GUARDED_BY(mutex); + +#ifdef OS_LINUX + Map TSA_GUARDED_BY(mutex) map; +/// std::shared_mutex may not be annotated with the 'capability' attribute in libcxx. +#else + Map map; +#endif }; } From 774cba09dfd4ab347d05caf45f8135a3a51771c3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 08:48:41 +0200 Subject: [PATCH 376/496] Fix flaky test_replicated_table_attach --- .../test_replicated_table_attach/configs/config.xml | 2 +- tests/integration/test_replicated_table_attach/test.py | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml index fea3eab4126..3f72f638776 100644 --- a/tests/integration/test_replicated_table_attach/configs/config.xml +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -1,6 +1,6 @@ 1 - 5 + 3 diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index de60b7ec291..4fe8064b26a 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -80,4 +80,8 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): assert_values() # check that we activate it in the end - node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + node.query_with_retry( + "INSERT INTO replicated_table_partitioned VALUES(20, 30)", + retry_count=20, + sleep_time=3, + ) From 797144270b3e20e9e4306949bde95c9a9a32c5e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 07:09:39 +0000 Subject: [PATCH 377/496] Update version_date.tsv and changelogs after v24.4.4.113-stable --- docs/changelogs/v24.4.4.113-stable.md | 73 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 74 insertions(+) create mode 100644 docs/changelogs/v24.4.4.113-stable.md diff --git a/docs/changelogs/v24.4.4.113-stable.md b/docs/changelogs/v24.4.4.113-stable.md new file mode 100644 index 00000000000..1f8a221a0a2 --- /dev/null +++ b/docs/changelogs/v24.4.4.113-stable.md @@ -0,0 +1,73 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.4.113-stable (d63a54957bd) FIXME as compared to v24.4.3.25-stable (a915dd4eda4) + +#### Improvement +* Backported in [#65884](https://github.com/ClickHouse/ClickHouse/issues/65884): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65303](https://github.com/ClickHouse/ClickHouse/issues/65303): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65894](https://github.com/ClickHouse/ClickHouse/issues/65894): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65372](https://github.com/ClickHouse/ClickHouse/issues/65372): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#66883](https://github.com/ClickHouse/ClickHouse/issues/66883): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65435](https://github.com/ClickHouse/ClickHouse/issues/65435): Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65448](https://github.com/ClickHouse/ClickHouse/issues/65448): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65710](https://github.com/ClickHouse/ClickHouse/issues/65710): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66689](https://github.com/ClickHouse/ClickHouse/issues/66689): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67499](https://github.com/ClickHouse/ClickHouse/issues/67499): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65353](https://github.com/ClickHouse/ClickHouse/issues/65353): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65060](https://github.com/ClickHouse/ClickHouse/issues/65060): Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.`. [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65329](https://github.com/ClickHouse/ClickHouse/issues/65329): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#64833](https://github.com/ClickHouse/ClickHouse/issues/64833): Fix bug which could lead to non-working TTLs with expressions. [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Backported in [#65086](https://github.com/ClickHouse/ClickHouse/issues/65086): Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65540](https://github.com/ClickHouse/ClickHouse/issues/65540): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65578](https://github.com/ClickHouse/ClickHouse/issues/65578): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65161](https://github.com/ClickHouse/ClickHouse/issues/65161): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65616](https://github.com/ClickHouse/ClickHouse/issues/65616): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65730](https://github.com/ClickHouse/ClickHouse/issues/65730): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65668](https://github.com/ClickHouse/ClickHouse/issues/65668): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65786](https://github.com/ClickHouse/ClickHouse/issues/65786): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65810](https://github.com/ClickHouse/ClickHouse/issues/65810): Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65931](https://github.com/ClickHouse/ClickHouse/issues/65931): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65826](https://github.com/ClickHouse/ClickHouse/issues/65826): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66299](https://github.com/ClickHouse/ClickHouse/issues/66299): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66326](https://github.com/ClickHouse/ClickHouse/issues/66326): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66153](https://github.com/ClickHouse/ClickHouse/issues/66153): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66459](https://github.com/ClickHouse/ClickHouse/issues/66459): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66224](https://github.com/ClickHouse/ClickHouse/issues/66224): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66267](https://github.com/ClickHouse/ClickHouse/issues/66267): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66678](https://github.com/ClickHouse/ClickHouse/issues/66678): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66603](https://github.com/ClickHouse/ClickHouse/issues/66603): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66358](https://github.com/ClickHouse/ClickHouse/issues/66358): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66971](https://github.com/ClickHouse/ClickHouse/issues/66971): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66968](https://github.com/ClickHouse/ClickHouse/issues/66968): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66719](https://github.com/ClickHouse/ClickHouse/issues/66719): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66950](https://github.com/ClickHouse/ClickHouse/issues/66950): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66947](https://github.com/ClickHouse/ClickHouse/issues/66947): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67631](https://github.com/ClickHouse/ClickHouse/issues/67631): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67195](https://github.com/ClickHouse/ClickHouse/issues/67195): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67377](https://github.com/ClickHouse/ClickHouse/issues/67377): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67240](https://github.com/ClickHouse/ClickHouse/issues/67240): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#67574](https://github.com/ClickHouse/ClickHouse/issues/67574): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65410](https://github.com/ClickHouse/ClickHouse/issues/65410): Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65903](https://github.com/ClickHouse/ClickHouse/issues/65903): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66385](https://github.com/ClickHouse/ClickHouse/issues/66385): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66424](https://github.com/ClickHouse/ClickHouse/issues/66424): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66542](https://github.com/ClickHouse/ClickHouse/issues/66542): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66857](https://github.com/ClickHouse/ClickHouse/issues/66857): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66873](https://github.com/ClickHouse/ClickHouse/issues/66873): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67057](https://github.com/ClickHouse/ClickHouse/issues/67057): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66944](https://github.com/ClickHouse/ClickHouse/issues/66944): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67250](https://github.com/ClickHouse/ClickHouse/issues/67250): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67410](https://github.com/ClickHouse/ClickHouse/issues/67410): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index b1391c2d781..7b5dcda82e3 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 +v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From dc65c0aa078cf06357291c0fe68f6c035698320f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 2 Aug 2024 07:15:40 +0000 Subject: [PATCH 378/496] Fix doc for parallel test execution Copy-pasterino strikes again. I forgot to remove the single quote. With it, pytest thinks the whole argument is a file: (no name '/ClickHouse/tests/integration/test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' in any of []) --- tests/integration/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index a8deb97b526..85146c79b1e 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -142,7 +142,7 @@ of parallel workers for `pytest-xdist`. $ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ $ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse $ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge -$ ./runner 'test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' +$ ./runner test_storage_s3_queue/test.py::test_max_set_age --count 10 -n 5 Start tests =============================================================================== test session starts ================================================================================ platform linux -- Python 3.10.12, pytest-7.4.4, pluggy-1.5.0 -- /usr/bin/python3 From 9c05a0ad5a0269af02ae2234e1d01dc3ce64bce2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 09:34:32 +0100 Subject: [PATCH 379/496] rm dirs in test_storage_delta --- tests/integration/test_storage_delta/test.py | 28 +++++++------------- 1 file changed, 10 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 384b8296f66..92a870ab360 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -8,6 +8,7 @@ import os import json import time import glob +import shutil import pyspark import delta @@ -52,15 +53,6 @@ def get_spark(): return builder.master("local").getOrCreate() -def remove_local_directory_contents(full_path): - for path in glob.glob(f"{full_path}/**"): - if os.path.isfile(path): - os.unlink(path) - else: - remove_local_directory_contents(path) - os.rmdir(path) - - @pytest.fixture(scope="module") def started_cluster(): try: @@ -179,7 +171,7 @@ def test_single_log_file(started_cluster): ) os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_partition_by(started_cluster): @@ -203,7 +195,7 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_checkpoint(started_cluster): @@ -280,7 +272,7 @@ def test_checkpoint(started_cluster): ).strip() ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") spark.sql(f"DROP TABLE {TABLE_NAME}") @@ -321,7 +313,7 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_metadata(started_cluster): @@ -357,7 +349,7 @@ def test_metadata(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_types(started_cluster): @@ -431,7 +423,7 @@ def test_types(started_cluster): ] ) - remove_local_directory_contents(f"/{result_file}") + shutil.rmtree(f"/{result_file}") spark.sql(f"DROP TABLE {TABLE_NAME}") @@ -496,7 +488,7 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_restart_broken_table_function(started_cluster): @@ -553,7 +545,7 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_partition_columns(started_cluster): @@ -753,5 +745,5 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini == 1 ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") spark.sql(f"DROP TABLE {TABLE_NAME}") From 01ca36cb5a157ab961dbd4460acc7e2ebb37e72a Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 09:37:47 +0100 Subject: [PATCH 380/496] empty From 27f4e1808e4cf299cd8eaf4a19c3bb979aa4e5bd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Aug 2024 10:43:02 +0200 Subject: [PATCH 381/496] Update tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh --- .../0_stateless/02434_cancel_insert_when_client_dies.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 45f4194104e..dca8dae22c3 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -10,10 +10,7 @@ export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE -$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A - settings non_replicated_deduplication_window=1000 - , merge_tree_clear_old_temporary_directories_interval_seconds = 1 - ;" +$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000, merge_tree_clear_old_temporary_directories_interval_seconds = 1;" $CLICKHOUSE_CLIENT -q "create table dedup_dist(A Int64) Engine = Distributed('test_cluster_one_shard_two_replicas', currentDatabase(), dedup_test)" function insert_data From 6c8f458b0bf9981068c7fecfdd9cef627406419b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 11:13:41 +0200 Subject: [PATCH 382/496] Fix reloading SQL UDFs with UNION --- .../UserDefinedSQLFunctionFactory.cpp | 8 +++++-- .../UserDefinedSQLObjectsDiskStorage.cpp | 6 ++--- .../UserDefinedSQLObjectsDiskStorage.h | 1 - .../UserDefinedSQLObjectsStorageBase.cpp | 15 +++++++++--- .../UserDefinedSQLObjectsStorageBase.h | 4 ++++ .../UserDefinedSQLObjectsZooKeeperStorage.cpp | 2 +- .../UserDefinedSQLObjectsZooKeeperStorage.h | 2 -- .../NormalizeSelectWithUnionQueryVisitor.h | 2 -- .../test.py | 23 +++++++++++++++++-- .../test.py | 12 ++++++++++ .../03215_udf_with_union.reference | 1 + .../0_stateless/03215_udf_with_union.sql | 14 +++++++++++ 12 files changed, 74 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03215_udf_with_union.reference create mode 100644 tests/queries/0_stateless/03215_udf_with_union.sql diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp index e6796874e50..d0bc812f91d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -9,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -80,13 +82,15 @@ namespace validateFunctionRecursiveness(*function_body, name); } - ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query) + ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query, const ContextPtr & context) { auto ptr = create_function_query.clone(); auto & res = typeid_cast(*ptr); res.if_not_exists = false; res.or_replace = false; FunctionNameNormalizer::visit(res.function_core.get()); + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(res.function_core); return ptr; } } @@ -125,7 +129,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeUnregistered(const ContextPtr & co bool UserDefinedSQLFunctionFactory::registerFunction(const ContextMutablePtr & context, const String & function_name, ASTPtr create_function_query, bool throw_if_exists, bool replace_if_exists) { checkCanBeRegistered(context, function_name, *create_function_query); - create_function_query = normalizeCreateFunctionQuery(*create_function_query); + create_function_query = normalizeCreateFunctionQuery(*create_function_query, context); try { diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp index 4c004d2537c..8910b45e79d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp @@ -1,7 +1,7 @@ #include "Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h" -#include "Functions/UserDefined/UserDefinedSQLFunctionFactory.h" -#include "Functions/UserDefined/UserDefinedSQLObjectType.h" +#include +#include #include #include @@ -54,7 +54,7 @@ namespace } UserDefinedSQLObjectsDiskStorage::UserDefinedSQLObjectsDiskStorage(const ContextPtr & global_context_, const String & dir_path_) - : global_context(global_context_) + : UserDefinedSQLObjectsStorageBase(global_context_) , dir_path{makeDirectoryPathCanonical(dir_path_)} , log{getLogger("UserDefinedSQLObjectsLoaderFromDisk")} { diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h index ae0cbd0c589..cafbd140598 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h @@ -42,7 +42,6 @@ private: ASTPtr tryLoadObject(UserDefinedSQLObjectType object_type, const String & object_name, const String & file_path, bool check_file_exists); String getFilePath(UserDefinedSQLObjectType object_type, const String & object_name) const; - ContextPtr global_context; String dir_path; LoggerPtr log; std::atomic objects_loaded = false; diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp index f251d11789f..225e919301d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp @@ -2,7 +2,10 @@ #include +#include +#include #include +#include #include namespace DB @@ -17,18 +20,24 @@ namespace ErrorCodes namespace { -ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query) +ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query, const ContextPtr & context) { auto ptr = create_function_query.clone(); auto & res = typeid_cast(*ptr); res.if_not_exists = false; res.or_replace = false; FunctionNameNormalizer::visit(res.function_core.get()); + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(res.function_core); return ptr; } } +UserDefinedSQLObjectsStorageBase::UserDefinedSQLObjectsStorageBase(ContextPtr global_context_) + : global_context(std::move(global_context_)) +{} + ASTPtr UserDefinedSQLObjectsStorageBase::get(const String & object_name) const { std::lock_guard lock(mutex); @@ -148,7 +157,7 @@ void UserDefinedSQLObjectsStorageBase::setAllObjects(const std::vector normalized_functions; for (const auto & [function_name, create_query] : new_objects) - normalized_functions[function_name] = normalizeCreateFunctionQuery(*create_query); + normalized_functions[function_name] = normalizeCreateFunctionQuery(*create_query, global_context); std::lock_guard lock(mutex); object_name_to_create_object_map = std::move(normalized_functions); @@ -166,7 +175,7 @@ std::vector> UserDefinedSQLObjectsStorageBase::getAllO void UserDefinedSQLObjectsStorageBase::setObject(const String & object_name, const IAST & create_object_query) { std::lock_guard lock(mutex); - object_name_to_create_object_map[object_name] = normalizeCreateFunctionQuery(create_object_query); + object_name_to_create_object_map[object_name] = normalizeCreateFunctionQuery(create_object_query, global_context); } void UserDefinedSQLObjectsStorageBase::removeObject(const String & object_name) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h index cab63a3bfcf..0dbc5586f08 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -13,6 +14,7 @@ namespace DB class UserDefinedSQLObjectsStorageBase : public IUserDefinedSQLObjectsStorage { public: + explicit UserDefinedSQLObjectsStorageBase(ContextPtr global_context_); ASTPtr get(const String & object_name) const override; ASTPtr tryGet(const String & object_name) const override; @@ -64,6 +66,8 @@ protected: std::unordered_map object_name_to_create_object_map; mutable std::recursive_mutex mutex; + + ContextPtr global_context; }; } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp index 01e7e3995fa..12c1302a3fe 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp @@ -48,7 +48,7 @@ namespace UserDefinedSQLObjectsZooKeeperStorage::UserDefinedSQLObjectsZooKeeperStorage( const ContextPtr & global_context_, const String & zookeeper_path_) - : global_context{global_context_} + : UserDefinedSQLObjectsStorageBase(global_context_) , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} , watch_queue{std::make_shared>>(std::numeric_limits::max())} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h index 61002be2bfd..0aa9b198398 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h @@ -68,8 +68,6 @@ private: void refreshObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); void syncObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); - ContextPtr global_context; - zkutil::ZooKeeperCachingGetter zookeeper_getter; String zookeeper_path; std::atomic objects_loaded = false; diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h index b2f55003da5..b642b5def91 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h @@ -4,8 +4,6 @@ #include #include -#include - namespace DB { diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py index e5f6683b90b..92d86a8fd2c 100644 --- a/tests/integration/test_replicated_user_defined_functions/test.py +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -141,6 +141,9 @@ def test_drop_if_exists(): def test_replication(): node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + node1.query( + "CREATE FUNCTION f3 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) assert ( node1.query("SELECT create_query FROM system.functions WHERE name='f2'") @@ -154,7 +157,11 @@ def test_replication(): assert node1.query("SELECT f2(12,3)") == "9\n" assert node2.query("SELECT f2(12,3)") == "9\n" + assert node1.query("SELECT f3()") == "2\n" + assert node2.query("SELECT f3()") == "2\n" + node1.query("DROP FUNCTION f2") + node1.query("DROP FUNCTION f3") assert ( node1.query("SELECT create_query FROM system.functions WHERE name='f2'") == "" ) @@ -214,7 +221,9 @@ def test_reload_zookeeper(): ) # config reloads, but can still work - node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + node1.query( + "CREATE FUNCTION f2 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) assert_eq_with_retry( node2, "SELECT name FROM system.functions WHERE name IN ['f1', 'f2'] ORDER BY name", @@ -269,7 +278,7 @@ def test_reload_zookeeper(): TSV(["f1", "f2", "f3"]), ) - assert node2.query("SELECT f1(12, 3), f2(12, 3), f3(12, 3)") == TSV([[15, 9, 4]]) + assert node2.query("SELECT f1(12, 3), f2(), f3(12, 3)") == TSV([[15, 2, 4]]) active_zk_connections = get_active_zk_connections() assert ( @@ -307,3 +316,13 @@ def test_start_without_zookeeper(): "CREATE FUNCTION f1 AS (x, y) -> (x + y)\n", ) node1.query("DROP FUNCTION f1") + + +def test_server_restart(): + node1.query( + "CREATE FUNCTION f1 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) + assert node1.query("SELECT f1()") == "2\n" + node1.restart_clickhouse() + assert node1.query("SELECT f1()") == "2\n" + node1.query("DROP FUNCTION f1") diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index 986438a4eed..bd491dfa195 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -18,20 +18,25 @@ def started_cluster(): def test_persistence(): create_function_query1 = "CREATE FUNCTION MySum1 AS (a, b) -> a + b" create_function_query2 = "CREATE FUNCTION MySum2 AS (a, b) -> MySum1(a, b) + b" + create_function_query3 = "CREATE FUNCTION MyUnion AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" instance.query(create_function_query1) instance.query(create_function_query2) + instance.query(create_function_query3) assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" + assert instance.query("SELECT MyUnion()") == "2\n" instance.restart_clickhouse() assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" + assert instance.query("SELECT MyUnion()") == "2\n" instance.query("DROP FUNCTION MySum2") instance.query("DROP FUNCTION MySum1") + instance.query("DROP FUNCTION MyUnion") instance.restart_clickhouse() @@ -48,3 +53,10 @@ def test_persistence(): or "Function with name 'MySum2' does not exist. In scope SELECT MySum2(1, 2)" in error_message ) + + error_message = instance.query_and_get_error("SELECT MyUnion()") + assert ( + "Unknown function MyUnion" in error_message + or "Function with name 'MyUnion' does not exist. In scope SELECT MyUnion" + in error_message + ) diff --git a/tests/queries/0_stateless/03215_udf_with_union.reference b/tests/queries/0_stateless/03215_udf_with_union.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03215_udf_with_union.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03215_udf_with_union.sql b/tests/queries/0_stateless/03215_udf_with_union.sql new file mode 100644 index 00000000000..00390c5d930 --- /dev/null +++ b/tests/queries/0_stateless/03215_udf_with_union.sql @@ -0,0 +1,14 @@ +DROP FUNCTION IF EXISTS 03215_udf_with_union; +CREATE FUNCTION 03215_udf_with_union AS () -> ( + SELECT sum(s) + FROM + ( + SELECT 1 AS s + UNION ALL + SELECT 1 AS s + ) +); + +SELECT 03215_udf_with_union(); + +DROP FUNCTION 03215_udf_with_union; From af53ed4c02ba52b3f57e97941b37a5931620d447 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Aug 2024 12:08:49 +0200 Subject: [PATCH 383/496] Ping CI From 62f0e09ecbb226ea72b5ee8d812436ef75038e33 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 12:17:08 +0200 Subject: [PATCH 384/496] Fix setting changes --- src/Core/SettingsChangesHistory.cpp | 264 +--------------------------- 1 file changed, 2 insertions(+), 262 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2438202f6a3..b6ef654438e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,268 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"restore_replace_external_table_functions_to_null", false, false, "New setting."}, - {"restore_replace_external_engines_to_null", false, false, "New setting."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - {"allow_archive_path_syntax", false, true, "Added new setting to allow disabling archive path syntax."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -338,6 +76,7 @@ static std::initializer_list Date: Mon, 29 Jul 2024 12:54:36 +0000 Subject: [PATCH 385/496] Trying to fix test_cache_evicted_by_temporary_data and print debug info --- .../config.d/storage_configuration.xml | 6 +-- .../test_temporary_data_in_cache/test.py | 44 ++++++++++++------- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml index 5a087d03266..107864fde0c 100644 --- a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -10,9 +10,9 @@ cache local_disk /tiny_local_cache/ - 10M - 1M - 1M + 12M + 100K + 100K 1 diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index cab134dcce2..abdfb5f4064 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -7,6 +7,9 @@ import fnmatch from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException + +MB = 1024 * 1024 + cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -36,15 +39,28 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(size) FROM system.filesystem_cache").strip() ) - assert get_cache_size() == 0 + def dump_debug_info(): + return "\n".join( + [ + ">>> filesystem_cache <<<", + q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), + ">>> remote_data_paths <<<", + q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), + ">>> tiny_local_cache_local_disk <<<", + q( + "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" + ), + ] + ) - assert get_free_space() > 8 * 1024 * 1024 + assert get_cache_size() == 0, dump_debug_info() + assert get_free_space() > 8 * MB, dump_debug_info() # Codec is NONE to make cache size predictable q( - "CREATE TABLE t1 (x UInt64 CODEC(NONE), y UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + "CREATE TABLE t1 (x UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" ) - q("INSERT INTO t1 SELECT number, number FROM numbers(1024 * 1024)") + q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") # To be sure that nothing is reading the cache and entries for t1 can be evited q("OPTIMIZE TABLE t1 FINAL") @@ -54,11 +70,11 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(x) FROM t1") cache_size_with_t1 = get_cache_size() - assert cache_size_with_t1 > 8 * 1024 * 1024 + assert cache_size_with_t1 > 8 * MB, dump_debug_info() # Almost all disk space is occupied by t1 cache free_space_with_t1 = get_free_space() - assert free_space_with_t1 < 4 * 1024 * 1024 + assert free_space_with_t1 < 4 * MB, dump_debug_info() # Try to sort the table, but fail because of lack of disk space with pytest.raises(QueryRuntimeException) as exc: @@ -76,31 +92,27 @@ def test_cache_evicted_by_temporary_data(start_cluster): # Some data evicted from cache by temporary data cache_size_after_eviction = get_cache_size() - assert cache_size_after_eviction < cache_size_with_t1 + assert cache_size_after_eviction < cache_size_with_t1, dump_debug_info() # Disk space freed, at least 3 MB, because temporary data tried to write 4 MB - assert get_free_space() > free_space_with_t1 + 3 * 1024 * 1024 + assert get_free_space() > free_space_with_t1 + 3 * MB, dump_debug_info() # Read some data to fill the cache again - q("SELECT avg(y) FROM t1") + q("SELECT avg(x) FROM t1") cache_size_with_t1 = get_cache_size() - assert cache_size_with_t1 > 8 * 1024 * 1024, q( - "SELECT * FROM system.filesystem_cache FORMAT Vertical" - ) + assert cache_size_with_t1 > 8 * MB, dump_debug_info() # Almost all disk space is occupied by t1 cache free_space_with_t1 = get_free_space() - assert free_space_with_t1 < 4 * 1024 * 1024, q( - "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" - ) + assert free_space_with_t1 < 4 * MB, dump_debug_info() node.http_query( "SELECT randomPrintableASCII(1024) FROM numbers(8 * 1024) FORMAT TSV", params={"buffer_size": 0, "wait_end_of_query": 1}, ) - assert get_free_space() > free_space_with_t1 + 3 * 1024 * 1024 + assert get_free_space() > free_space_with_t1 + 3 * MB, dump_debug_info() # not enough space for buffering 32 MB with pytest.raises(Exception) as exc: From 064c0eb9587d9dbd1fa81cdbae8554c22dd11734 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Fri, 2 Aug 2024 10:35:32 +0000 Subject: [PATCH 386/496] even better healthcheck for ldap --- tests/integration/compose/docker_compose_ldap.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/compose/docker_compose_ldap.yml b/tests/integration/compose/docker_compose_ldap.yml index 1f50b34735d..f49e00400a2 100644 --- a/tests/integration/compose/docker_compose_ldap.yml +++ b/tests/integration/compose/docker_compose_ldap.yml @@ -19,6 +19,7 @@ services: ldapsearch -x -H ldap://localhost:$$LDAP_PORT_NUMBER -D $$LDAP_ADMIN_DN -w $$LDAP_ADMIN_PASSWORD -b $$LDAP_ROOT | grep -c -E "member: cn=j(ohn|ane)doe" | grep 2 >> /dev/null + && cat /run/slapd/slapd.pid interval: 10s retries: 10 timeout: 2s From 02e48436057e45a884a1381e1c9cda9e1fe7de17 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 Jul 2024 09:30:39 +0000 Subject: [PATCH 387/496] test_cache_evicted_by_temporary_data drop cache --- .../test_temporary_data_in_cache/test.py | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index abdfb5f4064..87192a20975 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -39,19 +39,21 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(size) FROM system.filesystem_cache").strip() ) - def dump_debug_info(): - return "\n".join( - [ - ">>> filesystem_cache <<<", - q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), - ">>> remote_data_paths <<<", - q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), - ">>> tiny_local_cache_local_disk <<<", - q( - "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" - ), - ] - ) + dump_debug_info = lambda: "\n".join( + [ + ">>> filesystem_cache <<<", + q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), + ">>> remote_data_paths <<<", + q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), + ">>> tiny_local_cache_local_disk <<<", + q( + "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" + ), + ] + ) + + q("SYSTEM DROP FILESYSTEM CACHE") + q("DROP TABLE IF EXISTS t1 SYNC") assert get_cache_size() == 0, dump_debug_info() assert get_free_space() > 8 * MB, dump_debug_info() @@ -124,4 +126,4 @@ def test_cache_evicted_by_temporary_data(start_cluster): str(exc.value), "*Failed to reserve * for temporary file*" ), exc.value - q("DROP TABLE IF EXISTS t1") + q("DROP TABLE IF EXISTS t1 SYNC") From 092c837119a9be11cfcc85b4696e9a9c74d9bbc8 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 12:13:26 +0100 Subject: [PATCH 388/496] randomize table name in test_storage_delta --- tests/integration/test_storage_delta/test.py | 53 +++++++------------- 1 file changed, 17 insertions(+), 36 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 92a870ab360..054b79ff6fe 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -8,7 +8,8 @@ import os import json import time import glob -import shutil +import random +import string import pyspark import delta @@ -53,6 +54,11 @@ def get_spark(): return builder.master("local").getOrCreate() +def randomize_table_name(table_name, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{table_name}{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -152,7 +158,7 @@ def test_single_log_file(started_cluster): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_single_log_file" + TABLE_NAME = randomize_table_name("test_single_log_file") inserted_data = "SELECT number as a, toString(number + 1) as b FROM numbers(100)" parquet_data_path = create_initial_data_file( @@ -170,16 +176,13 @@ def test_single_log_file(started_cluster): inserted_data ) - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_partition_by(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_by" + TABLE_NAME = randomize_table_name("test_partition_by") write_delta_from_df( spark, @@ -195,15 +198,13 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 - shutil.rmtree(f"/{TABLE_NAME}") - def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_checkpoint" + TABLE_NAME = randomize_table_name("test_checkpoint") write_delta_from_df( spark, @@ -272,16 +273,13 @@ def test_checkpoint(started_cluster): ).strip() ) - shutil.rmtree(f"/{TABLE_NAME}") - spark.sql(f"DROP TABLE {TABLE_NAME}") - def test_multiple_log_files(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_log_files" + TABLE_NAME = randomize_table_name("test_multiple_log_files") write_delta_from_df( spark, generate_data(spark, 0, 100), f"/{TABLE_NAME}", mode="overwrite" @@ -313,15 +311,13 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) - shutil.rmtree(f"/{TABLE_NAME}") - def test_metadata(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata" + TABLE_NAME = randomize_table_name("test_metadata") parquet_data_path = create_initial_data_file( started_cluster, @@ -348,14 +344,11 @@ def test_metadata(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_types(started_cluster): - TABLE_NAME = "test_types" + TABLE_NAME = randomize_table_name("test_types") spark = started_cluster.spark_session - result_file = f"{TABLE_NAME}_result_2" + result_file = randomize_table_name(f"{TABLE_NAME}_result_2") delta_table = ( DeltaTable.create(spark) @@ -423,16 +416,13 @@ def test_types(started_cluster): ] ) - shutil.rmtree(f"/{result_file}") - spark.sql(f"DROP TABLE {TABLE_NAME}") - def test_restart_broken(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = "broken" - TABLE_NAME = "test_restart_broken" + TABLE_NAME = randomize_table_name("test_restart_broken") if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) @@ -487,16 +477,13 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_restart_broken_table_function(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = "broken2" - TABLE_NAME = "test_restart_broken_table_function" + TABLE_NAME = randomize_table_name("test_restart_broken_table_function") if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) @@ -544,16 +531,13 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_partition_columns(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_columns" + TABLE_NAME = randomize_table_name("test_partition_columns") result_file = f"{TABLE_NAME}" partition_columns = ["b", "c", "d", "e"] @@ -744,6 +728,3 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) - - shutil.rmtree(f"/{TABLE_NAME}") - spark.sql(f"DROP TABLE {TABLE_NAME}") From 7d1e958097e716f3ea1e0b7e51d6dfa575229c4c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 13:32:59 +0200 Subject: [PATCH 389/496] Integration tests: fix ports clashing problem --- tests/integration/conftest.py | 44 ++++++++++++++++++ tests/integration/helpers/cluster.py | 67 ++++++++++++++++++++++++---- 2 files changed, 103 insertions(+), 8 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index f4be31cc532..0a47840ede3 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -2,6 +2,8 @@ import logging import os +import socket +import multiprocessing import pytest # pylint:disable=import-error; for style check from helpers.cluster import run_and_check @@ -11,6 +13,7 @@ from helpers.network import _NetworkManager # # [1]: https://github.com/pytest-dev/pytest/issues/5502 logging.raiseExceptions = False +PORTS_PER_WORKER = 50 @pytest.fixture(scope="session", autouse=True) @@ -111,5 +114,46 @@ def pytest_addoption(parser): ) +def get_n_free_ports(total): + ports = [] + + while len(ports) < total: + with socket.socket() as s: + s.bind(("", 0)) + ports.append(s.getsockname()[1]) + + return ports + + def pytest_configure(config): os.environ["INTEGRATION_TESTS_RUN_ID"] = config.option.run_id + + # When running tests without pytest-xdist, + # the `pytest_xdist_setupnodes` hook is not executed + worker_ports = os.getenv("WORKER_FREE_PORTS", None) + if worker_ports is None: + os.environ["WORKER_FREE_PORTS"] = " ".join( + ([str(p) for p in get_n_free_ports(PORTS_PER_WORKER)]) + ) + + +def pytest_xdist_setupnodes(config, specs): + # Find {PORTS_PER_WORKER} * {number of xdist workers} ports and + # allocate pool of {PORTS_PER_WORKER} ports to each worker + + # Get number of xdist workers + num_workers = 1 + if os.environ.get("PYTEST_XDIST_WORKER", "master") == "master": + num_workers = config.getoption("numprocesses", 1) + if num_workers == "auto": + num_workers = multiprocessing.cpu_count() + + # Get free ports which will be distributed across workers + ports = get_n_free_ports(num_workers * PORTS_PER_WORKER) + + # Iterate over specs of workers and add allocated ports to env variable + for i, spec in enumerate(specs): + start_range = i * PORTS_PER_WORKER + spec.env["WORKER_FREE_PORTS"] = " ".join( + ([str(p) for p in ports[start_range : start_range + PORTS_PER_WORKER]]) + ) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6bc0ece63ca..3480a3089fe 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -135,6 +135,52 @@ def get_free_port(): return s.getsockname()[1] +def is_port_free(port: int) -> bool: + try: + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.bind(("", port)) + return True + except socket.error: + return False + + +class PortPoolManager: + """ + This class is used for distribution of ports allocated to single pytest-xdist worker + It can be used by multiple ClickHouseCluster instances + """ + + # Shared between instances + all_ports = None + free_ports = None + + def __init__(self): + self.used_ports = [] + + if self.all_ports is None: + worker_ports = os.getenv("WORKER_FREE_PORTS") + ports = [int(p) for p in worker_ports.split(" ")] + + # Static vars + PortPoolManager.all_ports = ports + PortPoolManager.free_ports = ports + + def get_port(self): + for port in self.free_ports: + if is_port_free(port): + self.free_ports.remove(port) + self.used_ports.append(port) + return port + + raise Exception( + f"No free ports: {self.all_ports}", + ) + + def return_used_ports(self): + self.free_ports.extend(self.used_ports) + self.used_ports.clear() + + def retry_exception(num, delay, func, exception=Exception, *args, **kwargs): """ Retry if `func()` throws, `num` times. @@ -716,62 +762,67 @@ class ClickHouseCluster: .stop() ) + self.port_pool = PortPoolManager() + @property def kafka_port(self): if self._kafka_port: return self._kafka_port - self._kafka_port = get_free_port() + self._kafka_port = self.port_pool.get_port() return self._kafka_port @property def schema_registry_port(self): if self._schema_registry_port: return self._schema_registry_port - self._schema_registry_port = get_free_port() + self._schema_registry_port = self.port_pool.get_port() return self._schema_registry_port @property def schema_registry_auth_port(self): if self._schema_registry_auth_port: return self._schema_registry_auth_port - self._schema_registry_auth_port = get_free_port() + self._schema_registry_auth_port = self.port_pool.get_port() return self._schema_registry_auth_port @property def kerberized_kafka_port(self): if self._kerberized_kafka_port: return self._kerberized_kafka_port - self._kerberized_kafka_port = get_free_port() + self._kerberized_kafka_port = self.port_pool.get_port() return self._kerberized_kafka_port @property def azurite_port(self): if self._azurite_port: return self._azurite_port - self._azurite_port = get_free_port() + self._azurite_port = self.port_pool.get_port() return self._azurite_port @property def mongo_port(self): if self._mongo_port: return self._mongo_port - self._mongo_port = get_free_port() + self._mongo_port = self.port_pool.get_port() return self._mongo_port @property def mongo_no_cred_port(self): if self._mongo_no_cred_port: return self._mongo_no_cred_port - self._mongo_no_cred_port = get_free_port() + self._mongo_no_cred_port = self.port_pool.get_port() return self._mongo_no_cred_port @property def redis_port(self): if self._redis_port: return self._redis_port - self._redis_port = get_free_port() + self._redis_port = self.port_pool.get_port() return self._redis_port + def __exit__(self, exc_type, exc_val, exc_tb): + self.port_pool.return_used_ports() + def print_all_docker_pieces(self): res_networks = subprocess.check_output( f"docker network ls --filter name='{self.project_name}*'", From 7d45529fe8d28a6b39deb32d060343bb5d03b64f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 12:35:40 +0100 Subject: [PATCH 390/496] randomize query id in test_checking_s3_blobs_paranoid --- .../test_checking_s3_blobs_paranoid/test.py | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index afe8449b44a..c22142046c7 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -4,6 +4,8 @@ import logging import pytest import os import minio +import random +import string from helpers.cluster import ClickHouseCluster from helpers.mock_servers import start_s3_mock @@ -45,6 +47,11 @@ def cluster(): cluster.shutdown() +def randomize_query_id(query_id, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{query_id}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + @pytest.fixture(scope="module") def init_broken_s3(cluster): yield start_s3_mock(cluster, "broken_s3", "8083") @@ -128,7 +135,7 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression broken_s3.setup_at_create_multi_part_upload() - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}") error = node.query_and_get_error( f""" INSERT INTO @@ -170,7 +177,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=1, after=2) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}") error = node.query_and_get_error( f""" INSERT INTO @@ -222,7 +229,7 @@ def test_when_error_is_retried(cluster, broken_s3, action_and_message): broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=3, after=2, action=action) - insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_{action}_RETRIED") node.query( f""" INSERT INTO @@ -251,7 +258,7 @@ def test_when_error_is_retried(cluster, broken_s3, action_and_message): assert s3_errors == 3 broken_s3.setup_at_part_upload(count=1000, after=2, action=action) - insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED_1" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_{action}_RETRIED_1") error = node.query_and_get_error( f""" INSERT INTO @@ -286,7 +293,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): action="broken_pipe", ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + insert_query_id = randomize_query_id(f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD") node.query( f""" INSERT INTO @@ -320,7 +327,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): after=2, action="broken_pipe", ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + insert_query_id = randomize_query_id(f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1") error = node.query_and_get_error( f""" INSERT INTO @@ -362,7 +369,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}" ) node.query( @@ -399,7 +406,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action="connection_reset_by_peer", action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}_1" ) error = node.query_and_get_error( @@ -444,7 +451,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}" ) node.query( @@ -482,7 +489,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}_1" ) error = node.query_and_get_error( @@ -522,7 +529,7 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): action="connection_refused", ) - insert_query_id = f"TEST_QUERY_IS_CANCELED_WITH_INF_RETRIES" + insert_query_id = randomize_query_id(f"TEST_QUERY_IS_CANCELED_WITH_INF_RETRIES") request = node.get_query_request( f""" INSERT INTO @@ -580,7 +587,7 @@ def test_adaptive_timeouts(cluster, broken_s3, node_name): count=1000000, ) - insert_query_id = f"TEST_ADAPTIVE_TIMEOUTS_{node_name}" + insert_query_id = randomize_query_id(f"TEST_ADAPTIVE_TIMEOUTS_{node_name}") node.query( f""" INSERT INTO From 34cba1bdda55cdd2409c535be56e4fe6165c894a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 11:46:03 +0000 Subject: [PATCH 391/496] Automatic style fix --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index c22142046c7..b995b4d6461 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -135,7 +135,9 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression broken_s3.setup_at_create_multi_part_upload() - insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}") + insert_query_id = randomize_query_id( + f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -177,7 +179,9 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=1, after=2) - insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}") + insert_query_id = randomize_query_id( + f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" + ) error = node.query_and_get_error( f""" INSERT INTO From d2d8a16ca6c5c3df31a62894fe2bcfb26d570061 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 13:51:03 +0200 Subject: [PATCH 392/496] Fix refreshable MVs --- src/Databases/DatabaseOrdinary.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 3ab5d3fa697..8808261654f 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -250,6 +251,8 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables convertMergeTreeToReplicatedIfNeeded(ast, qualified_name, file_name); + NormalizeSelectWithUnionQueryVisitor::Data data{local_context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); std::lock_guard lock{metadata.mutex}; metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast}; metadata.total_dictionaries += create_query->is_dictionary; From cae76458504f46fefe52c7d93594870d0bac4479 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:00:50 +0200 Subject: [PATCH 393/496] Make 02514_null_dictionary_source parallelizable --- tests/queries/0_stateless/02514_null_dictionary_source.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02514_null_dictionary_source.sql b/tests/queries/0_stateless/02514_null_dictionary_source.sql index 74fb57707ff..bfd36042f57 100644 --- a/tests/queries/0_stateless/02514_null_dictionary_source.sql +++ b/tests/queries/0_stateless/02514_null_dictionary_source.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP DICTIONARY IF EXISTS null_dict; CREATE DICTIONARY null_dict ( id UInt64, From 45f54c3633a388075898e3dcabc850dd1347587d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:01:01 +0200 Subject: [PATCH 394/496] Make 01086_window_view_cleanup parallelizable --- .../0_stateless/01086_window_view_cleanup.sh | 47 ++++++++++--------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 113bcffb2af..0bce08523e2 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel # Creation of a database with Ordinary engine emits a warning. CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal @@ -12,38 +11,40 @@ opts=( "--allow_experimental_analyzer=0" ) -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 < Date: Fri, 2 Aug 2024 14:03:27 +0200 Subject: [PATCH 395/496] Integration tests: fix ports clashing problem 2 --- tests/integration/conftest.py | 36 ++++++++++++++--------------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 0a47840ede3..a386ed53009 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -2,11 +2,9 @@ import logging import os -import socket -import multiprocessing import pytest # pylint:disable=import-error; for style check -from helpers.cluster import run_and_check +from helpers.cluster import run_and_check, is_port_free from helpers.network import _NetworkManager # This is a workaround for a problem with logging in pytest [1]. @@ -114,15 +112,16 @@ def pytest_addoption(parser): ) -def get_n_free_ports(total): +def get_unique_free_ports(total): ports = [] + for port in range(30000, 55000): + if is_port_free(port) and port not in ports: + ports.append(port) - while len(ports) < total: - with socket.socket() as s: - s.bind(("", 0)) - ports.append(s.getsockname()[1]) + if len(ports) == total: + return ports - return ports + raise Exception(f"Can't collect {total} ports. Collected: {len(ports)}") def pytest_configure(config): @@ -132,9 +131,8 @@ def pytest_configure(config): # the `pytest_xdist_setupnodes` hook is not executed worker_ports = os.getenv("WORKER_FREE_PORTS", None) if worker_ports is None: - os.environ["WORKER_FREE_PORTS"] = " ".join( - ([str(p) for p in get_n_free_ports(PORTS_PER_WORKER)]) - ) + master_ports = get_unique_free_ports(PORTS_PER_WORKER) + os.environ["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in master_ports])) def pytest_xdist_setupnodes(config, specs): @@ -142,18 +140,12 @@ def pytest_xdist_setupnodes(config, specs): # allocate pool of {PORTS_PER_WORKER} ports to each worker # Get number of xdist workers - num_workers = 1 - if os.environ.get("PYTEST_XDIST_WORKER", "master") == "master": - num_workers = config.getoption("numprocesses", 1) - if num_workers == "auto": - num_workers = multiprocessing.cpu_count() - + num_workers = len(specs) # Get free ports which will be distributed across workers - ports = get_n_free_ports(num_workers * PORTS_PER_WORKER) + ports = get_unique_free_ports(num_workers * PORTS_PER_WORKER) # Iterate over specs of workers and add allocated ports to env variable for i, spec in enumerate(specs): start_range = i * PORTS_PER_WORKER - spec.env["WORKER_FREE_PORTS"] = " ".join( - ([str(p) for p in ports[start_range : start_range + PORTS_PER_WORKER]]) - ) + per_workrer_ports = ports[start_range : start_range + PORTS_PER_WORKER] + spec.env["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in per_workrer_ports])) From 85cecf990ddefc25ff0903b6f97f84cd46dc471a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:06:51 +0200 Subject: [PATCH 396/496] Parallelize 02703_row_policies_for_database_combination --- ...3_row_policies_for_database_combination.sh | 92 +++++++++++++++++++ ..._row_policies_for_database_combination.sql | 88 ------------------ 2 files changed, 92 insertions(+), 88 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policies_for_database_combination.sh delete mode 100644 tests/queries/0_stateless/02703_row_policies_for_database_combination.sql diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh new file mode 100755 index 00000000000..f7b7c814d29 --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh @@ -0,0 +1,92 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery " + +DROP TABLE IF EXISTS 02703_rptable; +DROP TABLE IF EXISTS 02703_rptable_another; +CREATE TABLE 02703_rptable (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x; + +INSERT INTO 02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); + +CREATE TABLE 02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; + + +DROP ROW POLICY IF EXISTS 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; + +-- the test assumes users_without_row_policies_can_read_rows is true + +SELECT 'None'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; +SELECT 'R1: x == 1'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; +SELECT 'R1, R2: (x == 1) OR (x == 2)'; +SELECT * FROM 02703_rptable; + +SELECT 'R1, R2: (x == 2) FROM ANOTHER'; +SELECT * FROM 02703_rptable_another; + +CREATE ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; +SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; +SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; +SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +CREATE TABLE 02703_after_rp ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; +SELECT * FROM 02703_after_rp; + +-- does not matter if policies or table are created first +SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM AFTER_RP'; +SELECT * FROM 02703_after_rp; + +SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; +SELECT * FROM 02703_rptable_another; + +DROP ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R5: (x >= 2)'; +SELECT * FROM 02703_rptable; + +CREATE TABLE 02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree ORDER BY xx; +SELECT 'Policy not applicable'; +SELECT * FROM 02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query + +DROP ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +SELECT 'None'; +SELECT * FROM 02703_rptable; + +SELECT 'No problematic policy, select works'; +SELECT 'Ok' FROM ${CLICKHOUSE_DATABASE}.02703_unexpected_columns; + +DROP TABLE 02703_rptable; +DROP TABLE 02703_rptable_another; +DROP TABLE 02703_unexpected_columns; + +" diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql deleted file mode 100644 index 8c93fc595ba..00000000000 --- a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql +++ /dev/null @@ -1,88 +0,0 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS 02703_db; -CREATE DATABASE 02703_db; -DROP TABLE IF EXISTS 02703_db.02703_rptable; -DROP TABLE IF EXISTS 02703_db.02703_rptable_another; -CREATE TABLE 02703_db.02703_rptable (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x; - -INSERT INTO 02703_db.02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); - -CREATE TABLE 02703_db.02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_db.02703_rptable; - - -DROP ROW POLICY IF EXISTS 02703_filter_1 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_2 ON 02703_db.*; -DROP ROW POLICY IF EXISTS 02703_filter_3 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_4 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_5 ON 02703_db.*; - --- the test assumes users_without_row_policies_can_read_rows is true - -SELECT 'None'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_1 ON 02703_db.02703_rptable USING x=1 AS permissive TO ALL; -SELECT 'R1: x == 1'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_2 ON 02703_db.* USING x=2 AS permissive TO ALL; -SELECT 'R1, R2: (x == 1) OR (x == 2)'; -SELECT * FROM 02703_db.02703_rptable; - -SELECT 'R1, R2: (x == 2) FROM ANOTHER'; -SELECT * FROM 02703_db.02703_rptable_another; - -CREATE ROW POLICY 02703_filter_3 ON 02703_db.02703_rptable USING x=3 AS permissive TO ALL; -SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_4 ON 02703_db.02703_rptable USING x<=2 AS restrictive TO ALL; -SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_5 ON 02703_db.* USING y>=20 AS restrictive TO ALL; -SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE TABLE 02703_db.02703_after_rp ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_db.02703_rptable; -SELECT * FROM 02703_db.02703_after_rp; - --- does not matter if policies or table are created first -SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM AFTER_RP'; -SELECT * FROM 02703_db.02703_after_rp; - -SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; -SELECT * FROM 02703_db.02703_rptable_another; - -DROP ROW POLICY 02703_filter_1 ON 02703_db.02703_rptable; -SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_2 ON 02703_db.*; -SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_3 ON 02703_db.02703_rptable; -SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_4 ON 02703_db.02703_rptable; -SELECT 'R5: (x >= 2)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE TABLE 02703_db.02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree ORDER BY xx; -SELECT 'Policy not applicable'; -SELECT * FROM 02703_db.02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query - -DROP ROW POLICY 02703_filter_5 ON 02703_db.*; -SELECT 'None'; -SELECT * FROM 02703_db.02703_rptable; - -SELECT 'No problematic policy, select works'; -SELECT 'Ok' FROM 02703_db.02703_unexpected_columns; - -DROP TABLE 02703_db.02703_rptable; -DROP TABLE 02703_db.02703_rptable_another; -DROP TABLE 02703_db.02703_unexpected_columns; -DROP DATABASE 02703_db; From f55784c636c1dcb503dadb2f75bd6b586271bf0d Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Aug 2024 12:07:05 +0000 Subject: [PATCH 397/496] Fix 03203_client_benchmark_options --- tests/queries/0_stateless/03203_client_benchmark_options.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh index 37a1f2cd3ac..967db056c0b 100755 --- a/tests/queries/0_stateless/03203_client_benchmark_options.sh +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -5,10 +5,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh output=$(${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } +{ number=$(echo "$output" | grep -o "^[0-9]"); [[ -n "$number" && "$number" -ge 2 ]] && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } +{ number=$(echo "$output" | grep -o "^[0-9]"); [[ -n "$number" && "$number" -ge 2 ]] && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) { echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } From cb83274d37f8e90ef9c79451ce01a2ba95363158 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:18:31 +0200 Subject: [PATCH 398/496] Make 02703_row_policy_for_database parallelizable --- .../02703_row_policy_for_database.reference | 22 +++---- .../02703_row_policy_for_database.sh | 59 +++++++++++++++++++ .../02703_row_policy_for_database.sql | 53 ----------------- 3 files changed, 70 insertions(+), 64 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policy_for_database.sh delete mode 100644 tests/queries/0_stateless/02703_row_policy_for_database.sql diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.reference b/tests/queries/0_stateless/02703_row_policy_for_database.reference index b67ea69ae72..56b0d8c6f20 100644 --- a/tests/queries/0_stateless/02703_row_policy_for_database.reference +++ b/tests/queries/0_stateless/02703_row_policy_for_database.reference @@ -1,20 +1,20 @@ -- row policies for database - -- SHOW CREATE POLICY db1_02703 ON db1_02703.* -CREATE ROW POLICY db1_02703 ON db1_02703.* FOR SELECT USING 1 TO ALL - -- SHOW CREATE POLICY ON db1_02703.* -CREATE ROW POLICY db1_02703 ON db1_02703.* FOR SELECT USING 1 TO ALL -CREATE ROW POLICY tbl1_02703 ON db1_02703.`table` FOR SELECT USING 1 TO ALL - -- SHOW CREATE POLICY ON db1_02703.`*` + -- SHOW CREATE POLICY default ON default.* +CREATE ROW POLICY default_db_policy ON default.* FOR SELECT USING 1 TO ALL + -- SHOW CREATE POLICY ON default.* +CREATE ROW POLICY default_db_policy ON default.* FOR SELECT USING 1 TO ALL +CREATE ROW POLICY default_tb_policy ON default.`table` FOR SELECT USING 1 TO ALL + -- SHOW CREATE POLICY ON default.`*` R1, R2: (x == 1) OR (x == 2) 1 2 Check system.query_log SELECT \'-- row policies for database\'; [] -SELECT \' -- SHOW CREATE POLICY db1_02703 ON db1_02703.*\'; [] -SELECT \' -- SHOW CREATE POLICY ON db1_02703.*\'; [] -SELECT \' -- SHOW CREATE POLICY ON db1_02703.`*`\'; [] +SELECT \' -- SHOW CREATE POLICY default ON default.*\'; [] +SELECT \' -- SHOW CREATE POLICY ON default.*\'; [] +SELECT \' -- SHOW CREATE POLICY ON default.`*`\'; [] SELECT \'R1, R2: (x == 1) OR (x == 2)\'; [] -SELECT * FROM 02703_rqtable_default; ['`02703_filter_11_db` ON default.*','`02703_filter_11` ON default.`02703_rqtable_default`'] +SELECT * FROM 02703_rqtable_default; ['default_filter_11_db_policy ON default.*','default_filter_11_policy ON default.`02703_rqtable_default`'] SELECT \'Check system.query_log\'; [] -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE -CREATE ROW POLICY db2_02703 ON db1_02703.* TO u1_02703 +CREATE ROW POLICY db2_02703 ON default.* TO user_default diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.sh b/tests/queries/0_stateless/02703_row_policy_for_database.sh new file mode 100755 index 00000000000..e94bc7acd5e --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policy_for_database.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CLICKHOUSE_USER="user_$CLICKHOUSE_DATABASE" + +$CLICKHOUSE_CLIENT --multiquery " + +DROP USER IF EXISTS ${CLICKHOUSE_USER}; +CREATE USER ${CLICKHOUSE_USER}; + +CREATE TABLE ${CLICKHOUSE_DATABASE}.02703_rqtable (x UInt8) ENGINE = MergeTree ORDER BY x; +INSERT INTO ${CLICKHOUSE_DATABASE}.02703_rqtable VALUES (1), (2), (3), (4); + +SELECT '-- row policies for database'; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.* USING 1 AS PERMISSIVE TO ALL; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_tb_policy ON ${CLICKHOUSE_DATABASE}.table USING 1 AS PERMISSIVE TO ALL; +SELECT ' -- SHOW CREATE POLICY ${CLICKHOUSE_DATABASE} ON ${CLICKHOUSE_DATABASE}.*'; +SHOW CREATE POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.*; +SELECT ' -- SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.*'; +SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.*; +SELECT ' -- SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.\`*\`'; +SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.\`*\`; +DROP POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.*; +DROP POLICY ${CLICKHOUSE_DATABASE}_tb_policy ON ${CLICKHOUSE_DATABASE}.table; +" + +$CLICKHOUSE_CLIENT --query "CREATE ROW POLICY any_02703 ON *.some_table USING 1 AS PERMISSIVE TO ALL;" 2>&1 | grep -q "SYNTAX_ERROR" + +$CLICKHOUSE_CLIENT --multiquery " +CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; + +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON * USING x=1 AS permissive TO ALL; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_policy ON 02703_rqtable_default USING x=2 AS permissive TO ALL; + +INSERT INTO 02703_rqtable_default VALUES (1), (2), (3), (4); + +SELECT 'R1, R2: (x == 1) OR (x == 2)'; +SELECT * FROM 02703_rqtable_default; + +DROP TABLE 02703_rqtable_default; + +SELECT 'Check system.query_log'; +SYSTEM FLUSH LOGS; +SELECT query, used_row_policies FROM system.query_log WHERE current_database == currentDatabase() AND type == 'QueryStart' AND query_kind == 'Select' ORDER BY event_time_microseconds; + +DROP ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON *; +DROP ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_policy ON 02703_rqtable_default; + +USE ${CLICKHOUSE_DATABASE}; +SELECT ' -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE'; +CREATE ROW POLICY db2_02703 ON * TO ${CLICKHOUSE_USER}; +SHOW CREATE POLICY db2_02703 ON *; + +DROP ROW POLICY db2_02703 ON *; + +DROP USER ${CLICKHOUSE_USER}; +" diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.sql b/tests/queries/0_stateless/02703_row_policy_for_database.sql deleted file mode 100644 index 51ce5f4f870..00000000000 --- a/tests/queries/0_stateless/02703_row_policy_for_database.sql +++ /dev/null @@ -1,53 +0,0 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS db1_02703; -DROP USER IF EXISTS u1_02703; -CREATE USER u1_02703; - -CREATE DATABASE db1_02703; - -CREATE TABLE db1_02703.02703_rqtable (x UInt8) ENGINE = MergeTree ORDER BY x; -INSERT INTO db1_02703.02703_rqtable VALUES (1), (2), (3), (4); - - -SELECT '-- row policies for database'; -CREATE ROW POLICY db1_02703 ON db1_02703.* USING 1 AS PERMISSIVE TO ALL; -CREATE ROW POLICY tbl1_02703 ON db1_02703.table USING 1 AS PERMISSIVE TO ALL; -SELECT ' -- SHOW CREATE POLICY db1_02703 ON db1_02703.*'; -SHOW CREATE POLICY db1_02703 ON db1_02703.*; -SELECT ' -- SHOW CREATE POLICY ON db1_02703.*'; -SHOW CREATE POLICY ON db1_02703.*; -SELECT ' -- SHOW CREATE POLICY ON db1_02703.`*`'; -SHOW CREATE POLICY ON db1_02703.`*`; -DROP POLICY db1_02703 ON db1_02703.*; -DROP POLICY tbl1_02703 ON db1_02703.table; - -CREATE ROW POLICY any_02703 ON *.some_table USING 1 AS PERMISSIVE TO ALL; -- { clientError SYNTAX_ERROR } - -CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; - -CREATE ROW POLICY 02703_filter_11_db ON * USING x=1 AS permissive TO ALL; -CREATE ROW POLICY 02703_filter_11 ON 02703_rqtable_default USING x=2 AS permissive TO ALL; - -INSERT INTO 02703_rqtable_default VALUES (1), (2), (3), (4); - -SELECT 'R1, R2: (x == 1) OR (x == 2)'; -SELECT * FROM 02703_rqtable_default; - -DROP TABLE 02703_rqtable_default; - -SELECT 'Check system.query_log'; -SYSTEM FLUSH LOGS; -SELECT query, used_row_policies FROM system.query_log WHERE current_database == currentDatabase() AND type == 'QueryStart' AND query_kind == 'Select' ORDER BY event_time_microseconds; - -DROP ROW POLICY 02703_filter_11_db ON *; -DROP ROW POLICY 02703_filter_11 ON 02703_rqtable_default; - -USE db1_02703; -SELECT ' -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE'; -CREATE ROW POLICY db2_02703 ON * TO u1_02703; -SHOW CREATE POLICY db2_02703 ON *; - -DROP ROW POLICY db2_02703 ON *; - -DROP USER u1_02703; From 829d07c3a54716bb62cd36d4f2fa24da64ea6a0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:22:37 +0200 Subject: [PATCH 399/496] Make 02703_row_policies_for_asterisk parallelizable --- .../02703_row_policies_for_asterisk.sh | 14 +++++++++ .../02703_row_policies_for_asterisk.sql | 11 ------- ...3_row_policies_for_database_combination.sh | 30 +++++++++---------- 3 files changed, 29 insertions(+), 26 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policies_for_asterisk.sh delete mode 100644 tests/queries/0_stateless/02703_row_policies_for_asterisk.sql diff --git a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh new file mode 100755 index 00000000000..f9670e5f6f8 --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery " + SELECT 'Policy for table \`*\` does not affect other tables in the database'; + CREATE ROW POLICY 02703_asterisk_${CLICKHOUSE_DATABASE}_policy ON ${CLICKHOUSE_DATABASE}.\`*\` USING x=1 AS permissive TO ALL; + CREATE TABLE ${CLICKHOUSE_DATABASE}.\`*\` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; + CREATE TABLE ${CLICKHOUSE_DATABASE}.\`other\` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; + SELECT 'star', * FROM ${CLICKHOUSE_DATABASE}.\`*\`; + SELECT 'other', * FROM ${CLICKHOUSE_DATABASE}.other; + DROP ROW POLICY 02703_asterisk_${CLICKHOUSE_DATABASE}_policy ON ${CLICKHOUSE_DATABASE}.\`*\`; +" diff --git a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql deleted file mode 100644 index 96b1c01a6d6..00000000000 --- a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-parallel - -SELECT 'Policy for table `*` does not affect other tables in the database'; -CREATE DATABASE 02703_db_asterisk; -CREATE ROW POLICY 02703_asterisk ON 02703_db_asterisk.`*` USING x=1 AS permissive TO ALL; -CREATE TABLE 02703_db_asterisk.`*` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; -CREATE TABLE 02703_db_asterisk.`other` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; -SELECT 'star', * FROM 02703_db_asterisk.`*`; -SELECT 'other', * FROM 02703_db_asterisk.other; -DROP ROW POLICY 02703_asterisk ON 02703_db_asterisk.`*`; -DROP DATABASE 02703_db_asterisk; diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh index f7b7c814d29..35151eed220 100755 --- a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh +++ b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh @@ -14,37 +14,37 @@ INSERT INTO 02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); CREATE TABLE 02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; -DROP ROW POLICY IF EXISTS 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.*; -- the test assumes users_without_row_policies_can_read_rows is true SELECT 'None'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; SELECT 'R1: x == 1'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; SELECT 'R1, R2: (x == 1) OR (x == 2)'; SELECT * FROM 02703_rptable; SELECT 'R1, R2: (x == 2) FROM ANOTHER'; SELECT * FROM 02703_rptable_another; -CREATE ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; @@ -58,19 +58,19 @@ SELECT * FROM 02703_after_rp; SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; SELECT * FROM 02703_rptable_another; -DROP ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.*; SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R5: (x >= 2)'; SELECT * FROM 02703_rptable; @@ -78,7 +78,7 @@ CREATE TABLE 02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree OR SELECT 'Policy not applicable'; SELECT * FROM 02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query -DROP ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.*; SELECT 'None'; SELECT * FROM 02703_rptable; From 5256e8e6d08b5076be49b56fdebadb668892771c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 14:36:33 +0200 Subject: [PATCH 400/496] Integration tests: fix ports clashing problem 3 --- tests/integration/conftest.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index a386ed53009..b4c86a1cd2f 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -132,7 +132,7 @@ def pytest_configure(config): worker_ports = os.getenv("WORKER_FREE_PORTS", None) if worker_ports is None: master_ports = get_unique_free_ports(PORTS_PER_WORKER) - os.environ["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in master_ports])) + os.environ["WORKER_FREE_PORTS"] = " ".join([str(p) for p in master_ports]) def pytest_xdist_setupnodes(config, specs): @@ -148,4 +148,4 @@ def pytest_xdist_setupnodes(config, specs): for i, spec in enumerate(specs): start_range = i * PORTS_PER_WORKER per_workrer_ports = ports[start_range : start_range + PORTS_PER_WORKER] - spec.env["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in per_workrer_ports])) + spec.env["WORKER_FREE_PORTS"] = " ".join([str(p) for p in per_workrer_ports]) From db0bce33526abf16e705b9e56d178d6e2c45a36b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Aug 2024 15:01:15 +0200 Subject: [PATCH 401/496] Try make the code more understandable --- src/Interpreters/Cache/FileCache.cpp | 164 ++++++++++++------------- src/Interpreters/Cache/FileCache.h | 12 +- tests/config/config.d/storage_conf.xml | 2 +- 3 files changed, 80 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0a03f5dcc7d..4c17afb79be 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -318,7 +318,29 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + + /// Consider this example to understand why we need to account here for both `size` and `aligned_size`. + /// [________________]__________________] <-- requested range + /// ^ ^ + /// right offset aligned_right_offset + /// [_________] <-- last cached file segment, e.g. we have uncovered suffix of the requested range + /// [________________] + /// size + /// [____________________________________] + /// aligned_size + /// + /// So it is possible that we split this hole range into sub-segments by `max_file_segment_size` + /// and get something like this: + /// + /// [________________________] + /// ^ ^ + /// right_offset right_offset + max_file_segment_size + /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. + /// Because its left offset would be bigger than right_offset. + /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. + std::vector ranges; size_t current_pos = offset; @@ -339,42 +361,23 @@ std::vector FileCache::splitRange(size_t offset, size_t size return ranges; } -FileSegments FileCache::splitRangeIntoFileSegments( +FileSegments FileCache::createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - chassert(size > 0); - chassert(size <= aligned_size); - /// We take `size` as a soft limit and `aligned_size` as a hard limit. - - auto current_pos = offset; - auto end_pos_non_included = offset + size; - - size_t current_file_segment_size; - size_t remaining_size = aligned_size; - - FileSegments file_segments; - const size_t max_size = max_file_segment_size.load(); - while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit)) + FileSegments result; + for (const auto & r : ranges) { - current_file_segment_size = std::min(remaining_size, max_size); - remaining_size -= current_file_segment_size; - - auto file_segment_metadata_it = addFileSegment( - locked_key, current_pos, current_file_segment_size, state, create_settings, nullptr); - file_segments.push_back(file_segment_metadata_it->second->file_segment); - - current_pos += current_file_segment_size; + if (file_segments_limit && file_segments_count >= file_segments_limit) + break; + auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); + result.push_back(metadata_it->second->file_segment); + ++file_segments_count; } - - chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), - fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); - return file_segments; + return result; } void FileCache::fillHolesWithEmptyFileSegments( @@ -448,18 +451,9 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); } if (is_limit_reached()) @@ -493,29 +487,20 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; - auto non_aligned_size = non_aligned_right_offset - current_pos + 1; + auto non_aligned_hole_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { auto file_segment = std::make_shared( - locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, create_settings); + locked_key.getKey(), current_pos, non_aligned_hole_size, FileSegment::State::DETACHED, create_settings); file_segments.insert(file_segments.end(), file_segment); } else { - auto ranges = splitRange(current_pos, non_aligned_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, non_aligned_hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); if (is_limit_reached()) erase_unprocessed(); @@ -548,8 +533,9 @@ FileSegmentsHolderPtr FileCache::set( } else { - file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + const auto ranges = splitRange(offset, size, size); + size_t file_segments_count = 0; + file_segments = createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -569,23 +555,27 @@ FileCache::getOrSet( assertInitialized(); - FileSegment::Range range(offset, offset + size - 1); + FileSegment::Range initial_range(offset, offset + size - 1); + /// result_range is initial range, which will be adjusted according to + /// 1. aligned offset, alighed_end_offset + /// 2. max_file_segments_limit + FileSegment::Range result_range = initial_range; - const auto aligned_offset = roundDownToMultiple(range.left, boundary_alignment); - auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; + const auto aligned_offset = roundDownToMultiple(initial_range.left, boundary_alignment); + auto aligned_end_offset = std::min(roundUpToMultiple(initial_range.right + 1, boundary_alignment), file_size) - 1; - chassert(aligned_offset <= range.left); - chassert(aligned_end_offset >= range.right); + chassert(aligned_offset <= initial_range.left); + chassert(aligned_end_offset >= initial_range.right); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user); /// Get all segments which intersect with the given range. - auto file_segments = getImpl(*locked_key, range, file_segments_limit); + auto file_segments = getImpl(*locked_key, initial_range, file_segments_limit); if (file_segments_limit) { chassert(file_segments.size() <= file_segments_limit); if (file_segments.size() == file_segments_limit) - range.right = aligned_end_offset = file_segments.back()->range().right; + result_range.right = aligned_end_offset = file_segments.back()->range().right; } /// Check case if we have uncovered prefix, e.g. @@ -597,11 +587,11 @@ FileCache::getOrSet( /// [ ] /// ^----^ /// uncovered prefix. - const bool has_uncovered_prefix = file_segments.empty() || range.left < file_segments.front()->range().left; + const bool has_uncovered_prefix = file_segments.empty() || result_range.left < file_segments.front()->range().left; - if (aligned_offset < range.left && has_uncovered_prefix) + if (aligned_offset < result_range.left && has_uncovered_prefix) { - auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? range.left - 1 : file_segments.front()->range().left - 1); + auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? result_range.left - 1 : file_segments.front()->range().left - 1); auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); if (prefix_file_segments.empty()) @@ -610,7 +600,7 @@ FileCache::getOrSet( /// ^ ^ ^ /// aligned_offset range.left range.right /// [___] [__________] <-- current cache (example) - range.left = aligned_offset; + result_range.left = aligned_offset; } else { @@ -621,10 +611,10 @@ FileCache::getOrSet( /// ^ /// prefix_file_segments.back().right - chassert(prefix_file_segments.back()->range().right < range.left); + chassert(prefix_file_segments.back()->range().right < result_range.left); chassert(prefix_file_segments.back()->range().right >= aligned_offset); - range.left = prefix_file_segments.back()->range().right + 1; + result_range.left = prefix_file_segments.back()->range().right + 1; } } @@ -637,11 +627,11 @@ FileCache::getOrSet( /// [___] /// ^---^ /// uncovered_suffix - const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < range.right; + const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < result_range.right; - if (range.right < aligned_end_offset && has_uncovered_suffix) + if (result_range.right < aligned_end_offset && has_uncovered_suffix) { - auto suffix_range = FileSegment::Range(range.right, aligned_end_offset); + auto suffix_range = FileSegment::Range(result_range.right, aligned_end_offset); /// We need to get 1 file segment, so file_segments_limit = 1 here. auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); @@ -652,7 +642,7 @@ FileCache::getOrSet( /// range.left range.right aligned_end_offset /// [___] [___] <-- current cache (example) - range.right = aligned_end_offset; + result_range.right = aligned_end_offset; } else { @@ -662,35 +652,33 @@ FileCache::getOrSet( /// [___] [___] [_________] <-- current cache (example) /// ^ /// suffix_file_segments.front().left - range.right = suffix_file_segments.front()->range().left - 1; + result_range.right = suffix_file_segments.front()->range().left - 1; } } - chassert(range.left >= aligned_offset); - if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments( - *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), - FileSegment::State::EMPTY, file_segments_limit, create_settings); + auto ranges = splitRange(result_range.left, initial_range.size() + (initial_range.left - result_range.left), result_range.size()); + size_t file_segments_count = file_segments.size(); + file_segments.splice(file_segments.end(), createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, file_segments_limit, create_settings)); } else { - chassert(file_segments.front()->range().right >= range.left); - chassert(file_segments.back()->range().left <= range.right); + chassert(file_segments.front()->range().right >= result_range.left); + chassert(file_segments.back()->range().left <= result_range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, result_range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(range.left)) + if (!file_segments.front()->range().contains(result_range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", - file_segments.front()->range().toString(), offset, range.right, aligned_offset, aligned_end_offset); + file_segments.front()->range().toString(), offset, result_range.right, aligned_offset, aligned_end_offset); } } - chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); + chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 3f7eec73b56..07be802a940 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -265,16 +265,10 @@ private: /// each subrange size must be less or equal to max_file_segment_size. std::vector splitRange(size_t offset, size_t size, size_t aligned_size); - /// Split range into subranges by max_file_segment_size (same as in splitRange()) - /// and create a new file segment for each subrange. - /// If `file_segments_limit` > 0, create no more than first file_segments_limit - /// file segments. - FileSegments splitRangeIntoFileSegments( + FileSegments createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 4daa64b520d..e106e3a0e6b 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,7 @@ cache s3_disk s3_cache/ - 100Mi + 104857600 5Mi 1 100 From 8d979680060f10e6bcec3fc83fb3bdbaa7bb3deb Mon Sep 17 00:00:00 2001 From: Lennard Eijsackers Date: Fri, 2 Aug 2024 15:13:37 +0200 Subject: [PATCH 402/496] Use FunctionArgumentDescriptors to check bitSlice function + add test case Signed-off-by: Lennard Eijsackers --- src/Functions/bitSlice.cpp | 30 ++++++++----------- ...214_bitslice_argument_evaluation.reference | 0 .../03214_bitslice_argument_evaluation.sql | 10 +++++++ 3 files changed, 22 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference create mode 100644 tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql diff --git a/src/Functions/bitSlice.cpp b/src/Functions/bitSlice.cpp index e2b455846d8..f1d3bb57221 100644 --- a/src/Functions/bitSlice.cpp +++ b/src/Functions/bitSlice.cpp @@ -40,28 +40,22 @@ public: bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const size_t number_of_arguments = arguments.size(); + FunctionArgumentDescriptors mandatory_args{ + {"s", static_cast(&isStringOrFixedString), nullptr, "String"}, + {"offset", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + }; - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), number_of_arguments); + FunctionArgumentDescriptors optional_args{ + {"length", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + }; - if (!isString(arguments[0]) && !isStringOrFixedString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - if (arguments[0]->onlyNull()) - return arguments[0]; + validateFunctionArguments(*this, arguments, mandatory_args, optional_args); - if (!isNativeNumber(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", - arguments[1]->getName(), getName()); - - if (number_of_arguments == 3 && !isNativeNumber(arguments[2])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", - arguments[2]->getName(), getName()); + const auto & type = arguments[0].type; + if (type->onlyNull()) + return type; return std::make_shared(); } diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql new file mode 100644 index 00000000000..b8488600fcb --- /dev/null +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql @@ -0,0 +1,10 @@ +-- No arguments passed +SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Invalid 1st argument passed +SELECT bitSlice(1, 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument, invalid 2nd argument passed +SELECT bitSlice('Hello', 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument & 2nd argument, invalid 3rd argument passed +SELECT bitSlice('Hello', 1, 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- More arguments then expected +SELECT bitSlice('Hello', 1, 1, 'World'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 83096249a7480b0bfa1d9246c17136727bba904c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 13:54:32 +0000 Subject: [PATCH 403/496] Update version_date.tsv and changelogs after v24.3.6.48-lts --- docs/changelogs/v24.3.6.48-lts.md | 39 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 40 insertions(+) create mode 100644 docs/changelogs/v24.3.6.48-lts.md diff --git a/docs/changelogs/v24.3.6.48-lts.md b/docs/changelogs/v24.3.6.48-lts.md new file mode 100644 index 00000000000..f045afc619b --- /dev/null +++ b/docs/changelogs/v24.3.6.48-lts.md @@ -0,0 +1,39 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.6.48-lts (b2d33c3c45d) FIXME as compared to v24.3.5.46-lts (fe54cead6b6) + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#66889](https://github.com/ClickHouse/ClickHouse/issues/66889): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66687](https://github.com/ClickHouse/ClickHouse/issues/66687): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67497](https://github.com/ClickHouse/ClickHouse/issues/67497): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#66324](https://github.com/ClickHouse/ClickHouse/issues/66324): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66151](https://github.com/ClickHouse/ClickHouse/issues/66151): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66451](https://github.com/ClickHouse/ClickHouse/issues/66451): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66222](https://github.com/ClickHouse/ClickHouse/issues/66222): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66676](https://github.com/ClickHouse/ClickHouse/issues/66676): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66602](https://github.com/ClickHouse/ClickHouse/issues/66602): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66356](https://github.com/ClickHouse/ClickHouse/issues/66356): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66970](https://github.com/ClickHouse/ClickHouse/issues/66970): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66967](https://github.com/ClickHouse/ClickHouse/issues/66967): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66718](https://github.com/ClickHouse/ClickHouse/issues/66718): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66949](https://github.com/ClickHouse/ClickHouse/issues/66949): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66946](https://github.com/ClickHouse/ClickHouse/issues/66946): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67629](https://github.com/ClickHouse/ClickHouse/issues/67629): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67193](https://github.com/ClickHouse/ClickHouse/issues/67193): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67375](https://github.com/ClickHouse/ClickHouse/issues/67375): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67572](https://github.com/ClickHouse/ClickHouse/issues/67572): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66422](https://github.com/ClickHouse/ClickHouse/issues/66422): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66855](https://github.com/ClickHouse/ClickHouse/issues/66855): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#67055](https://github.com/ClickHouse/ClickHouse/issues/67055): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66943](https://github.com/ClickHouse/ClickHouse/issues/66943): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 7b5dcda82e3..24488066190 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -10,6 +10,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.6.48-lts 2024-08-02 v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 From 9c7464e0653782af385dbc884dd3acecfc69c6cc Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:04:11 +0200 Subject: [PATCH 404/496] Stateless tests: reduce pure_http_client timeout to get reasons of timed out tests --- tests/queries/0_stateless/helpers/pure_http_client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 0e7a4d27f4f..7a8efec36bb 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -18,7 +18,7 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=1500, settings=dict(), binary_result=False + self, query, connection_timeout=500, settings=dict(), binary_result=False ): NUMBER_OF_TRIES = 30 DELAY = 10 @@ -47,12 +47,12 @@ class ClickHouseClient: else: raise ValueError(r.text) - def query_return_df(self, query, connection_timeout=1500): + def query_return_df(self, query, connection_timeout=500): data = self.query(query, connection_timeout) df = pd.read_csv(io.StringIO(data), sep="\t") return df - def query_with_data(self, query, data, connection_timeout=1500, settings=dict()): + def query_with_data(self, query, data, connection_timeout=500, settings=dict()): params = { "query": query, "timeout_before_checking_execution_speed": 120, From 2c9cef38e56c65ec9bbe7f3af21d4865662f6e9a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:05:39 +0200 Subject: [PATCH 405/496] Stateless tests: fix hanging tests `02473_multistep_prewhere*` `00411_long_accurate_number_comparison*` --- ...411_long_accurate_number_comparison.python | 19 ++++++------------- .../02473_multistep_prewhere.python | 4 ++-- .../02473_multistep_split_prewhere.python | 4 ++-- .../0_stateless/helpers/pure_http_client.py | 17 ++++++++++++++++- 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index 045de9ee7ee..38b108a696f 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -2,23 +2,16 @@ import os, itertools, urllib.request, urllib.parse, urllib.error, urllib.request, urllib.error, urllib.parse, sys +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) -def get_ch_answer(query): - return ( - urllib.request.urlopen( - os.environ.get( - "CLICKHOUSE_URL", - "http://localhost:" + os.environ.get("CLICKHOUSE_PORT_HTTP", "8123"), - ), - data=query.encode(), - ) - .read() - .decode() - ) +from pure_http_client import ClickHouseClient + +client = ClickHouseClient() def check_answers(query, answer): - ch_answer = get_ch_answer(query) + ch_answer = client.query(query) if ch_answer.strip() != answer.strip(): print("FAIL on query:", query) print("Expected answer:", answer) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index 11095202039..09326b6365d 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -195,7 +195,7 @@ def main(): default_index_granularity = 10 total_rows = 7 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [ default_index_granularity - 1, default_index_granularity, diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.python b/tests/queries/0_stateless/02473_multistep_split_prewhere.python index 19444994fd2..10e94059171 100644 --- a/tests/queries/0_stateless/02473_multistep_split_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -161,7 +161,7 @@ def main(): default_index_granularity = 10 total_rows = 8 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [default_index_granularity - 1, default_index_granularity]: tester = Tester(session, url, index_granularity, total_rows) # Test combinations of ranges of columns c and d diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 7a8efec36bb..a31a91e0550 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -1,7 +1,8 @@ import os import io -import sys import requests +from requests.adapters import HTTPAdapter +from requests.packages.urllib3.util.retry import Retry import time import pandas as pd @@ -77,3 +78,17 @@ class ClickHouseClient: return result else: raise ValueError(r.text) + + +def requests_session_with_retries(retries=3, timeout=180): + session = requests.Session() + retry = Retry( + total=retries, + read=retries, + connect=retries, + ) + adapter = HTTPAdapter(max_retries=retry) + session.mount("http://", adapter) + session.mount("https://", adapter) + session.timeout = timeout + return session From aefed7cdd62e874f7507afe69d803c9164a283ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Aug 2024 16:06:53 +0200 Subject: [PATCH 406/496] Update tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql --- .../0_stateless/03164_s3_settings_for_queries_and_merges.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index 001ef382850..a6932e0536c 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -21,6 +21,7 @@ SYSTEM DROP MARK CACHE; SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; +-- Errors in S3 requests will be automatically retried, however ProfileEvents can be wrong. That is why we subtract errors. SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 From 664e131f4f2e46fc216305c440e840a5a5784328 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:09:48 +0200 Subject: [PATCH 407/496] Integration tests: fix ports clashing problem 4 --- tests/integration/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index b4c86a1cd2f..aa235118aed 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -1,4 +1,6 @@ #!/usr/bin/env python3 +# pylint: disable=unused-argument +# pylint: disable=broad-exception-raised import logging import os From b3e2ce695514d4d314ed8ac1ecdb111c5f94ac7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 408/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 2110b765d19ce4e68d0c23eab219e199aec0ea96 Mon Sep 17 00:00:00 2001 From: Lennard Eijsackers Date: Fri, 2 Aug 2024 17:53:12 +0200 Subject: [PATCH 409/496] Style check fix + adding debug info to query output Signed-off-by: Lennard Eijsackers --- src/Functions/bitSlice.cpp | 2 -- .../03214_bitslice_argument_evaluation.reference | 11 +++++++++++ .../03214_bitslice_argument_evaluation.sql | 1 + 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Functions/bitSlice.cpp b/src/Functions/bitSlice.cpp index f1d3bb57221..f24473351ae 100644 --- a/src/Functions/bitSlice.cpp +++ b/src/Functions/bitSlice.cpp @@ -18,9 +18,7 @@ using namespace GatherUtils; namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionBitSlice : public IFunction diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference index e69de29bb2d..1731dfa0d79 100644 --- a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference @@ -0,0 +1,11 @@ +-- { echo } +-- No arguments passed +SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Invalid 1st argument passed +SELECT bitSlice(1, 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument, invalid 2nd argument passed +SELECT bitSlice('Hello', 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument & 2nd argument, invalid 3rd argument passed +SELECT bitSlice('Hello', 1, 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- More arguments then expected +SELECT bitSlice('Hello', 1, 1, 'World'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql index b8488600fcb..1731dfa0d79 100644 --- a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql @@ -1,3 +1,4 @@ +-- { echo } -- No arguments passed SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Invalid 1st argument passed From 364622f567028ffc70785b681fc246d7151eef04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 410/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From fcb0ce7361f74dd8d97a3007f77248f293b2ce5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 18:17:32 +0200 Subject: [PATCH 411/496] Fix docs build --- docker/docs/builder/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/docs/builder/run.sh b/docker/docs/builder/run.sh index 01c15cb4b0f..d73adb5d279 100755 --- a/docker/docs/builder/run.sh +++ b/docker/docs/builder/run.sh @@ -26,7 +26,6 @@ sed -i '/onBrokenMarkdownLinks:/ s/ignore/error/g' docusaurus.config.js if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then export CI=true - yarn install exec yarn build "$@" fi From cebb3668380f65187b201e638013df40f8ac8ada Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 09:23:40 +0200 Subject: [PATCH 412/496] more fixes --- .github/actions/check_workflow/action.yml | 21 ++++ .github/workflows/create_release.yml | 27 +++-- .github/workflows/pull_request.yml | 9 +- tests/ci/artifactory.py | 6 +- tests/ci/auto_release.py | 6 +- tests/ci/ci.py | 4 +- tests/ci/ci_buddy.py | 30 +++-- tests/ci/ci_cache.py | 14 +-- tests/ci/ci_config.py | 3 +- tests/ci/ci_definitions.py | 3 +- tests/ci/ci_metadata.py | 4 +- tests/ci/ci_utils.py | 30 ++++- tests/ci/create_release.py | 131 ++++++++++++++-------- tests/ci/docker_server.py | 73 +++--------- tests/ci/test_docker.py | 46 +------- 15 files changed, 213 insertions(+), 194 deletions(-) create mode 100644 .github/actions/check_workflow/action.yml diff --git a/.github/actions/check_workflow/action.yml b/.github/actions/check_workflow/action.yml new file mode 100644 index 00000000000..19a3cec76f5 --- /dev/null +++ b/.github/actions/check_workflow/action.yml @@ -0,0 +1,21 @@ +name: CheckWorkflowResults + +description: Check overall workflow status and post error to slack if any + +inputs: + needs: + description: github needs context as a json string + required: true + type: string + +runs: + using: "composite" + steps: + - name: Check Workflow + shell: bash + run: | + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ inputs.needs }} + EOF + python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index e27db1b09a4..29094cc51a6 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -24,7 +24,7 @@ concurrency: dry-run: description: 'Dry run' required: false - default: true + default: false type: boolean jobs: @@ -43,16 +43,27 @@ jobs: - name: Prepare Release Info shell: bash run: | + if [ ${{ inputs.only-repo }} == "true" ]; then + git tag -l ${{ inputs.ref }} || { echo "With only-repo option ref must be a valid release tag"; exit 1; } + fi python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ + ${{ inputs.dry-run == true && '--dry-run' || '' }} \ + ${{ inputs.only-repo == true && '--skip-tag-check' || '' }} echo "::group::Release Info" python3 -m json.tool /tmp/release_info.json echo "::endgroup::" release_tag=$(jq -r '.release_tag' /tmp/release_info.json) commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + is_latest=$(jq -r '.latest' /tmp/release_info.json) echo "Release Tag: $release_tag" echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + if [ "$is_latest" == "true" ]; then + echo "DOCKER_TAG_TYPE=release-latest" >> "$GITHUB_ENV" + else + echo "DOCKER_TAG_TYPE=release" >> "$GITHUB_ENV" + fi - name: Download All Release Artifacts if: ${{ inputs.type == 'patch' }} shell: bash @@ -85,10 +96,11 @@ jobs: echo "Generate ChangeLog" export CI=1 docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --volume=".:/wd" --workdir="/wd" \ + clickhouse/style-test \ + ./tests/ci/changelog.py -v --debug-helpers \ --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + --output="./docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md echo "Generate Security" python3 ./utils/security-generator/generate_security.py > SECURITY.md @@ -160,7 +172,7 @@ jobs: cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker server release" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building if: ${{ inputs.type == 'patch' }} @@ -169,7 +181,7 @@ jobs: cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker keeper release" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed - name: Update release info. Merge created PRs shell: bash @@ -178,6 +190,7 @@ jobs: - name: Set current Release progress to Completed with OK shell: bash run: | + # dummy stage to finalize release info with "progress: completed; status: OK" python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" python3 ./tests/ci/create_release.py --set-progress-completed - name: Post Slack Message diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 04bef1460a6..071f0f1e20a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -172,12 +172,9 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} - name: Check Workflow results - run: | - export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat > "$WORKFLOW_RESULT_FILE" << 'EOF' - ${{ toJson(needs) }} - EOF - python3 ./tests/ci/ci_buddy.py --check-wf-status + uses: ./.github/actions/check_workflow + with: + needs: ${{ toJson(needs) }} ################################# Stage Final ################################# # diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 8bba7bca30e..f3d7d24f717 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -158,7 +158,7 @@ class DebianArtifactory: print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) - self.release_info.debian_command = debian_command + self.release_info.debian = debian_command self.release_info.dump() @@ -240,7 +240,7 @@ class RpmArtifactory: print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) - self.release_info.rpm_command = rpm_command + self.release_info.rpm = rpm_command self.release_info.dump() @@ -304,7 +304,7 @@ class TgzArtifactory: expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" Shell.check("rm /tmp/tmp.tgz*", verbose=True) - self.release_info.tgz_command = cmd + self.release_info.tgz = cmd self.release_info.dump() diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 6c17b4c74ad..3cc88634004 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -127,15 +127,13 @@ def _prepare(token): ) commit_num -= 1 - is_completed = CI.GHActions.check_wf_completed( - token=token, commit_sha=commit - ) + is_completed = CI.GH.check_wf_completed(token=token, commit_sha=commit) if not is_completed: print(f"CI is in progress for [{commit}] - check previous commit") commits_to_branch_head += 1 continue - commit_ci_status = CI.GHActions.get_commit_status_by_name( + commit_ci_status = CI.GH.get_commit_status_by_name( token=token, commit_sha=commit, status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 935fe472e50..2565c8944e4 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -16,7 +16,7 @@ import upload_result_helper from build_check import get_release_or_pr from ci_config import CI from ci_metadata import CiMetadata -from ci_utils import GHActions, normalize_string, Utils +from ci_utils import GH, normalize_string, Utils from clickhouse_helper import ( CiLogsCredentials, ClickHouseHelper, @@ -368,7 +368,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): ) to_be_skipped = True # skip_status = SUCCESS already there - GHActions.print_in_group("Commit Status Data", job_status) + GH.print_in_group("Commit Status Data", job_status) # create pre report jr = JobReport.create_pre_report(status=skip_status, job_skipped=to_be_skipped) diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index 138909c1db0..f0e73e925fe 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -8,7 +8,7 @@ import requests from botocore.exceptions import ClientError from pr_info import PRInfo -from ci_utils import Shell, GHActions +from ci_config import CI class CIBuddy: @@ -31,10 +31,19 @@ class CIBuddy: self.sha = pr_info.sha[:10] def check_workflow(self): - GHActions.print_workflow_results() - res = GHActions.get_workflow_job_result(GHActions.ActionsNames.RunConfig) - if res != GHActions.ActionStatuses.SUCCESS: - self.post_job_error("Workflow Configuration Failed", critical=True) + CI.GH.print_workflow_results() + if CI.Envs.GITHUB_WORKFLOW == CI.WorkFlowNames.CreateRelease: + if not CI.GH.is_workflow_ok(): + self.post_job_error( + f"{CI.Envs.GITHUB_WORKFLOW} Workflow Failed", critical=True + ) + else: + res = CI.GH.get_workflow_job_result(CI.GH.ActionsNames.RunConfig) + if res != CI.GH.ActionStatuses.SUCCESS: + print(f"ERROR: RunConfig status is [{res}] - post report to slack") + self.post_job_error( + f"{CI.Envs.GITHUB_WORKFLOW} Workflow Failed", critical=True + ) @staticmethod def _get_webhooks(): @@ -74,10 +83,13 @@ class CIBuddy: message = title if isinstance(body, dict): for name, value in body.items(): - if "commit_sha" in name: + if "sha" in name and value and len(value) == 40: value = ( f"" ) + elif isinstance(value, str) and value.startswith("https://github.com/"): + value_shorten = value.split("/")[-1] + value = f"<{value}|{value_shorten}>" message += f" *{name}*: {value}\n" else: message += body + "\n" @@ -120,9 +132,11 @@ class CIBuddy: ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: - instance_id = Shell.get_output("ec2metadata --instance-id") or instance_id + instance_id = ( + CI.Shell.get_output("ec2metadata --instance-id") or instance_id + ) instance_type = ( - Shell.get_output("ec2metadata --instance-type") or instance_type + CI.Shell.get_output("ec2metadata --instance-type") or instance_type ) if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 85eabb84f9f..4846233ab03 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -7,7 +7,7 @@ from typing import Dict, Optional, Any, Union, Sequence, List, Set from ci_config import CI -from ci_utils import is_hex, GHActions +from ci_utils import is_hex, GH from commit_status_helper import CommitStatusData from env_helper import ( TEMP_PATH, @@ -258,15 +258,15 @@ class CiCache: def print_status(self): print(f"Cache enabled: [{self.enabled}]") for record_type in self.RecordType: - GHActions.print_in_group( + GH.print_in_group( f"Cache records: [{record_type}]", list(self.records[record_type]) ) - GHActions.print_in_group( + GH.print_in_group( "Jobs to do:", list(self.jobs_to_do.items()), ) - GHActions.print_in_group("Jobs to skip:", self.jobs_to_skip) - GHActions.print_in_group( + GH.print_in_group("Jobs to skip:", self.jobs_to_skip) + GH.print_in_group( "Jobs to wait:", list(self.jobs_to_wait.items()), ) @@ -788,7 +788,7 @@ class CiCache: while round_cnt < MAX_ROUNDS_TO_WAIT: round_cnt += 1 - GHActions.print_in_group( + GH.print_in_group( f"Wait pending jobs, round [{round_cnt}/{MAX_ROUNDS_TO_WAIT}]:", list(self.jobs_to_wait), ) @@ -853,7 +853,7 @@ class CiCache: # make up for 2 iterations in dry_run expired_sec += int(TIMEOUT / 2) + 1 - GHActions.print_in_group( + GH.print_in_group( "Remaining jobs:", [list(self.jobs_to_wait)], ) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a7df884a091..c031ca9b805 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -34,7 +34,8 @@ class CI: from ci_definitions import Runners as Runners from ci_utils import Envs as Envs from ci_utils import Utils as Utils - from ci_utils import GHActions as GHActions + from ci_utils import GH as GH + from ci_utils import Shell as Shell from ci_definitions import Labels as Labels from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS from ci_definitions import WorkFlowNames as WorkFlowNames diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 51de8c63509..de6791acda8 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -112,6 +112,7 @@ class WorkFlowNames(metaclass=WithIter): """ JEPSEN = "JepsenWorkflow" + CreateRelease = "CreateRelease" class BuildNames(metaclass=WithIter): @@ -578,7 +579,7 @@ class CommonJobConfigs: DOCKER_SERVER = JobConfig( job_name_keyword="docker", required_on_release_branch=True, - run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', + run_command='docker_server.py --check-name "$CHECK_NAME" --tag-type head --allow-build-reuse', digest=DigestConfig( include_paths=[ "tests/ci/docker_server.py", diff --git a/tests/ci/ci_metadata.py b/tests/ci/ci_metadata.py index a767d102811..67106262634 100644 --- a/tests/ci/ci_metadata.py +++ b/tests/ci/ci_metadata.py @@ -9,7 +9,7 @@ from env_helper import ( S3_BUILDS_BUCKET_PUBLIC, ) from s3_helper import S3Helper -from ci_utils import GHActions +from ci_utils import GH from synchronizer_utils import SYNC_BRANCH_PREFIX @@ -111,7 +111,7 @@ class CiMetadata: else: log_title = f"Storing workflow metadata: PR [{self.pr_number}], upstream PR [{self.upstream_pr_number}]" - GHActions.print_in_group( + GH.print_in_group( log_title, [f"run_id: {self.run_id}"], ) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 4f696a2c55a..dae1520afb6 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -16,6 +16,8 @@ class Envs: WORKFLOW_RESULT_FILE = os.getenv( "WORKFLOW_RESULT_FILE", "/tmp/workflow_results.json" ) + S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") + GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") LABEL_CATEGORIES = { @@ -83,7 +85,7 @@ def normalize_string(string: str) -> str: return res -class GHActions: +class GH: class ActionsNames: RunConfig = "RunConfig" @@ -117,6 +119,14 @@ class GHActions: results = [f"{job}: {data['result']}" for job, data in res.items()] cls.print_in_group("Workflow results", results) + @classmethod + def is_workflow_ok(cls) -> bool: + res = cls._get_workflow_results() + for _job, data in res.items(): + if data["result"] == "failure": + return False + return bool(res) + @classmethod def get_workflow_job_result(cls, wf_job_name: str) -> Optional[str]: res = cls._get_workflow_results() @@ -189,15 +199,25 @@ class GHActions: return False @staticmethod - def get_pr_url_by_branch(repo, branch): - get_url_cmd = ( - f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" - ) + def get_pr_url_by_branch(branch, repo=None): + repo = repo or Envs.GITHUB_REPOSITORY + get_url_cmd = f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url' --state open" url = Shell.get_output(get_url_cmd) + if not url: + print(f"WARNING: No open PR found, branch [{branch}] - search for merged") + get_url_cmd = f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url' --state merged" + url = Shell.get_output(get_url_cmd) if not url: print(f"ERROR: PR nor found, branch [{branch}]") return url + @staticmethod + def is_latest_release_branch(branch): + latest_branch = Shell.get_output( + 'gh pr list --label release --repo ClickHouse/ClickHouse --search "sort:created" -L1 --json headRefName' + ) + return latest_branch == branch + class Shell: @classmethod diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index c407a74fbf0..b4e08f29dbe 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -10,9 +10,8 @@ from typing import Iterator, List from git_helper import Git, GIT_PREFIX from ssh import SSHAgent -from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper -from ci_utils import Shell, GHActions +from ci_utils import Shell, GH from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, @@ -69,13 +68,14 @@ class ReleaseContextManager: previous_release_tag="NA", previous_release_sha="NA", release_progress=ReleaseProgress.STARTED, + latest=False, ).dump() else: # fetch release info from fs and update self.release_info = ReleaseInfo.from_file() assert self.release_info assert ( - self.release_info.progress_description == ReleaseProgressDescription.OK + self.release_info.progress_status == ReleaseProgressDescription.OK ), "Must be OK on the start of new context" self.release_info.release_progress = self.release_progress self.release_info.dump() @@ -84,9 +84,9 @@ class ReleaseContextManager: def __exit__(self, exc_type, exc_value, traceback): assert self.release_info if exc_type is not None: - self.release_info.progress_description = ReleaseProgressDescription.FAILED + self.release_info.progress_status = ReleaseProgressDescription.FAILED else: - self.release_info.progress_description = ReleaseProgressDescription.OK + self.release_info.progress_status = ReleaseProgressDescription.OK self.release_info.dump() @@ -96,6 +96,7 @@ class ReleaseInfo: release_tag: str release_branch: str commit_sha: str + latest: bool # lts or stable codename: str previous_release_tag: str @@ -104,12 +105,12 @@ class ReleaseInfo: version_bump_pr: str = "" prs_merged: bool = False release_url: str = "" - debian_command: str = "" - rpm_command: str = "" - tgz_command: str = "" - docker_command: str = "" + debian: str = "" + rpm: str = "" + tgz: str = "" + docker: str = "" release_progress: str = "" - progress_description: str = "" + progress_status: str = "" def is_patch(self): return self.release_branch != "master" @@ -129,12 +130,15 @@ class ReleaseInfo: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) return self - def prepare(self, commit_ref: str, release_type: str) -> "ReleaseInfo": + def prepare( + self, commit_ref: str, release_type: str, skip_tag_check: bool + ) -> "ReleaseInfo": version = None release_branch = None release_tag = None previous_release_tag = None previous_release_sha = None + latest_release = False codename = "" assert release_type in ("patch", "new") if release_type == "new": @@ -145,7 +149,7 @@ class ReleaseInfo: verbose=True, ) with checkout(commit_ref): - commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") + commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -158,12 +162,12 @@ class ReleaseInfo: release_tag = version.describe previous_release_tag = expected_prev_tag previous_release_sha = Shell.get_output_or_raise( - f"git rev-parse {previous_release_tag}" + f"git rev-list -n1 {previous_release_tag}" ) assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") + commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -200,16 +204,20 @@ class ReleaseInfo: expected_tag_prefix ) and git.latest_tag.endswith(expected_tag_suffix): pass - else: + elif not skip_tag_check: assert ( False - ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]" + ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" previous_release_sha = Shell.get_output_or_raise( - f"git rev-parse {previous_release_tag}" + f"git rev-list -n1 {previous_release_tag}" ) assert previous_release_sha + if CI.GH.is_latest_release_branch(release_branch): + print("This is going to be the latest release!") + latest_release = True + assert ( release_branch and previous_release_tag @@ -218,7 +226,7 @@ class ReleaseInfo: and release_tag and version and (codename in ("lts", "stable") or release_type == "new") - ) + ), f"Check: {release_branch}, {previous_release_tag}, {previous_release_sha}, {commit_sha}, {release_tag}, {version}" self.release_branch = release_branch self.commit_sha = commit_sha @@ -228,7 +236,8 @@ class ReleaseInfo: self.previous_release_tag = previous_release_tag self.previous_release_sha = previous_release_sha self.release_progress = ReleaseProgress.STARTED - self.progress_description = ReleaseProgressDescription.OK + self.progress_status = ReleaseProgressDescription.OK + self.latest = latest_release return self def push_release_tag(self, dry_run: bool) -> None: @@ -252,7 +261,7 @@ class ReleaseInfo: @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: - cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" + cmd = f"gh api repos/{CI.Envs.GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" Shell.check(cmd, dry_run=dry_run, strict=True) def push_new_release_branch(self, dry_run: bool) -> None: @@ -294,7 +303,7 @@ class ReleaseInfo: f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) Shell.check( - f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' + f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, @@ -303,9 +312,12 @@ class ReleaseInfo: verbose=True, ) + def get_version_bump_branch(self): + return f"bump_version_{self.version}" + def update_version_and_contributors_list(self, dry_run: bool) -> None: # Bump version, update contributors list, create PR - branch_upd_version_contributors = f"bump_version_{self.version}" + branch_upd_version_contributors = self.get_version_bump_branch() with checkout(self.commit_sha): git = Git() version = get_version_from_repo(git=git) @@ -323,9 +335,9 @@ class ReleaseInfo: update_contributors(raise_error=True) cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" - body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" - cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" + body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" + cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body \"{body}\" --assignee {actor}" Shell.check( cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True ) @@ -342,30 +354,42 @@ class ReleaseInfo: ) self.version_bump_pr = "dry-run" else: - self.version_bump_pr = GHActions.get_pr_url_by_branch( - repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors + self.version_bump_pr = GH.get_pr_url_by_branch( + branch=branch_upd_version_contributors ) + def get_change_log_branch(self): + return f"auto/{self.release_tag}" + def update_release_info(self, dry_run: bool) -> "ReleaseInfo": if self.release_branch != "master": - branch = f"auto/{release_info.release_tag}" - if not dry_run: - url = GHActions.get_pr_url_by_branch( - repo=GITHUB_REPOSITORY, branch=branch - ) - else: - url = "dry-run" - print(f"ChangeLog PR url [{url}]") - self.changelog_pr = url - print(f"Release url [{url}]") - self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" - if self.release_progress == ReleaseProgress.COMPLETED: - self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" + if not self.changelog_pr: + branch = self.get_change_log_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"ChangeLog PR url [{url}]") + self.changelog_pr = url + + if not self.version_bump_pr: + branch = self.get_version_bump_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"Version bump PR url [{url}]") + self.version_bump_pr = url + + self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + print(f"Release url [{self.release_url}]") + + self.docker = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" self.dump() return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: - repo = os.getenv("GITHUB_REPOSITORY") + repo = CI.Envs.GITHUB_REPOSITORY assert repo cmds = [ f"gh release create --repo {repo} --title 'Release {self.release_tag}' {self.release_tag}" @@ -375,7 +399,9 @@ class ReleaseInfo: if not dry_run: for cmd in cmds: Shell.check(cmd, strict=True, verbose=True) - self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + self.release_url = ( + f"https://github.com/{repo}/releases/tag/{self.release_tag}" + ) else: print("Dry-run, would run commands:") print("\n * ".join(cmds)) @@ -536,7 +562,7 @@ class PackageDownloader: ] ) self.s3.download_file( - bucket=S3_BUILDS_BUCKET, + bucket=CI.Envs.S3_BUILDS_BUCKET, s3_path=s3_path, local_file_path="/".join([self.LOCAL_DIR, package_file]), ) @@ -557,7 +583,7 @@ class PackageDownloader: ] ) self.s3.download_file( - bucket=S3_BUILDS_BUCKET, + bucket=CI.Envs.S3_BUILDS_BUCKET, s3_path=s3_path, local_file_path="/".join([self.LOCAL_DIR, destination_binary_name]), ) @@ -636,6 +662,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Initial step to prepare info like release branch, release tag, etc.", ) + parser.add_argument( + "--skip-tag-check", + action="store_true", + help="To skip check against latest git tag on a release branch", + ) parser.add_argument( "--push-release-tag", action="store_true", @@ -725,7 +756,11 @@ if __name__ == "__main__": assert ( args.ref and args.release_type ), "--ref and --release-type must be provided with --prepare-release-info" - release_info.prepare(commit_ref=args.ref, release_type=args.release_type) + release_info.prepare( + commit_ref=args.ref, + release_type=args.release_type, + skip_tag_check=args.skip_tag_check, + ) if args.download_packages: with ReleaseContextManager( @@ -776,7 +811,7 @@ if __name__ == "__main__": else: title = "New release" if ( - release_info.progress_description == ReleaseProgressDescription.OK + release_info.progress_status == ReleaseProgressDescription.OK and release_info.release_progress == ReleaseProgress.COMPLETED ): title = "Completed: " + title @@ -792,16 +827,16 @@ if __name__ == "__main__": if args.set_progress_started: ri = ReleaseInfo.from_file() ri.release_progress = args.progress - ri.progress_description = ReleaseProgressDescription.FAILED + ri.progress_status = ReleaseProgressDescription.FAILED ri.dump() assert args.progress, "Progress step name must be provided" if args.set_progress_completed: ri = ReleaseInfo.from_file() assert ( - ri.progress_description == ReleaseProgressDescription.FAILED + ri.progress_status == ReleaseProgressDescription.FAILED ), "Must be FAILED before set to OK" - ri.progress_description = ReleaseProgressDescription.OK + ri.progress_status = ReleaseProgressDescription.OK ri.dump() if args.merge_prs: diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 3e782c079c6..8f0474d5053 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -69,13 +69,14 @@ def parse_args() -> argparse.Namespace: help="sha of the commit to use packages from", ) parser.add_argument( - "--release-type", + "--tag-type", type=str, - choices=("auto", "latest", "major", "minor", "patch", "head"), + choices=("head", "release", "latest-release"), default="head", - help="version part that will be updated when '--version' is set; " - "'auto' is a special case, it will get versions from github and detect the " - "release type (latest, major, minor or patch) automatically", + help="defines required tags for resulting docker image. " + "head - for master image (tag: head) " + "release - for release image (tags: XX, XX.XX, XX.XX.XX, XX.XX.XX.XX) " + "release-latest - for latest release image (tags: XX, XX.XX, XX.XX.XX, XX.XX.XX.XX, latest) ", ) parser.add_argument( "--image-path", @@ -149,74 +150,35 @@ def retry_popen(cmd: str, log_file: Path) -> int: return retcode -def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: - if release_type != "auto": - return release_type - - git_versions = get_tagged_versions() - reference_version = git_versions[0] - for i in reversed(range(len(git_versions))): - if git_versions[i] <= version: - if i == len(git_versions) - 1: - return "latest" - reference_version = git_versions[i + 1] - break - - if version.major < reference_version.major: - return "major" - if version.minor < reference_version.minor: - return "minor" - if version.patch < reference_version.patch: - return "patch" - - raise ValueError( - "Release type 'tweak' is not supported for " - f"{version.string} < {reference_version.string}" - ) - - -def gen_tags(version: ClickHouseVersion, release_type: str) -> List[str]: +def gen_tags(version: ClickHouseVersion, tag_type: str) -> List[str]: """ - 22.2.2.2 + latest: + @tag_type release-latest, @version 22.2.2.2: - latest - 22 - 22.2 - 22.2.2 - 22.2.2.2 - 22.2.2.2 + major: + @tag_type release, @version 22.2.2.2: - 22 - 22.2 - 22.2.2 - 22.2.2.2 - 22.2.2.2 + minor: - - 22.2 - - 22.2.2 - - 22.2.2.2 - 22.2.2.2 + patch: - - 22.2.2 - - 22.2.2.2 - 22.2.2.2 + head: + @tag_type head: - head """ parts = version.string.split(".") tags = [] - if release_type == "latest": - tags.append(release_type) + if tag_type == "release-latest": + tags.append("latest") for i in range(len(parts)): tags.append(".".join(parts[: i + 1])) - elif release_type == "major": + elif tag_type == "head": + tags.append(tag_type) + elif tag_type == "release": for i in range(len(parts)): tags.append(".".join(parts[: i + 1])) - elif release_type == "minor": - for i in range(1, len(parts)): - tags.append(".".join(parts[: i + 1])) - elif release_type == "patch": - for i in range(2, len(parts)): - tags.append(".".join(parts[: i + 1])) - elif release_type == "head": - tags.append(release_type) else: - raise ValueError(f"{release_type} is not valid release part") + assert False, f"Invalid release type [{tag_type}]" return tags @@ -370,8 +332,7 @@ def main(): push = True image = DockerImageData(image_path, image_repo, False) - args.release_type = auto_release_type(args.version, args.release_type) - tags = gen_tags(args.version, args.release_type) + tags = gen_tags(args.version, args.tag_type) repo_urls = {} direct_urls: Dict[str, List[str]] = {} diff --git a/tests/ci/test_docker.py b/tests/ci/test_docker.py index 662143bfd9b..58ebe4ecbb1 100644 --- a/tests/ci/test_docker.py +++ b/tests/ci/test_docker.py @@ -1,61 +1,19 @@ #!/usr/bin/env python import unittest -from unittest.mock import patch, MagicMock from version_helper import get_version_from_string import docker_server as ds -# di.logging.basicConfig(level=di.logging.INFO) - class TestDockerServer(unittest.TestCase): def test_gen_tags(self): version = get_version_from_string("22.2.2.2") cases = ( - ("latest", ["latest", "22", "22.2", "22.2.2", "22.2.2.2"]), - ("major", ["22", "22.2", "22.2.2", "22.2.2.2"]), - ("minor", ["22.2", "22.2.2", "22.2.2.2"]), - ("patch", ["22.2.2", "22.2.2.2"]), + ("release-latest", ["latest", "22", "22.2", "22.2.2", "22.2.2.2"]), + ("release", ["22", "22.2", "22.2.2", "22.2.2.2"]), ("head", ["head"]), ) for case in cases: release_type = case[0] self.assertEqual(case[1], ds.gen_tags(version, release_type)) - - with self.assertRaises(ValueError): - ds.gen_tags(version, "auto") - - @patch("docker_server.get_tagged_versions") - def test_auto_release_type(self, mock_tagged_versions: MagicMock) -> None: - mock_tagged_versions.return_value = [ - get_version_from_string("1.1.1.1"), - get_version_from_string("1.2.1.1"), - get_version_from_string("2.1.1.1"), - get_version_from_string("2.2.1.1"), - get_version_from_string("2.2.2.1"), - ] - - cases_less = ( - (get_version_from_string("1.0.1.1"), "minor"), - (get_version_from_string("1.1.2.1"), "minor"), - (get_version_from_string("1.3.1.1"), "major"), - (get_version_from_string("2.1.2.1"), "minor"), - (get_version_from_string("2.2.1.3"), "patch"), - (get_version_from_string("2.2.3.1"), "latest"), - (get_version_from_string("2.3.1.1"), "latest"), - ) - for case in cases_less: - release = ds.auto_release_type(case[0], "auto") - self.assertEqual(case[1], release) - - cases_equal = ( - (get_version_from_string("1.1.1.1"), "minor"), - (get_version_from_string("1.2.1.1"), "major"), - (get_version_from_string("2.1.1.1"), "minor"), - (get_version_from_string("2.2.1.1"), "patch"), - (get_version_from_string("2.2.2.1"), "latest"), - ) - for case in cases_equal: - release = ds.auto_release_type(case[0], "auto") - self.assertEqual(case[1], release) From ae6d4bdd8aa2b010f27dd8ebbb6f816362eb2a9e Mon Sep 17 00:00:00 2001 From: Sasha Sheikin Date: Mon, 29 Jul 2024 10:31:35 +0200 Subject: [PATCH 413/496] Fix positionCaseInsensitive example --- .../en/sql-reference/functions/string-search-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index b7ba1d4feb7..e9ff7ebf33b 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -150,15 +150,15 @@ A case insensitive invariant of [position](#position). Query: ``` sql -SELECT position('Hello, world!', 'hello'); +SELECT positionCaseInsensitive('Hello, world!', 'hello'); ``` Result: ``` text -┌─position('Hello, world!', 'hello')─┐ -│ 0 │ -└────────────────────────────────────┘ +┌─positionCaseInsensitive('Hello, world!', 'hello')─┐ +│ 1 │ +└───────────────────────────────────────────────────┘ ``` ## positionUTF8 From 89c47df559ba23d988f8af3c342e0c8d5531f4b8 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Aug 2024 16:52:46 +0000 Subject: [PATCH 414/496] Fix variant as common type in if function with Tuples and Maps --- src/Functions/if.cpp | 10 ++++++++++ .../03215_varian_as_common_type_tuple_map.reference | 10 ++++++++++ .../03215_varian_as_common_type_tuple_map.sql | 7 +++++++ 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 07dbee27a9d..64da6e95a43 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -632,6 +632,11 @@ private: ColumnPtr executeTuple(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { + /// For different Tuples the result type can be Variant with this Tuples if use_variant_as_common_type=1. + /// In this case we should use generic implementation. + if (!isTuple(result_type)) + return nullptr; + /// Calculate function for each corresponding elements of tuples. const ColumnWithTypeAndName & arg1 = arguments[1]; @@ -677,6 +682,11 @@ private: ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { + /// For different Maps the result type can be Variant with this Maps if use_variant_as_common_type=1. + /// In this case we should use generic implementation. + if (!isMap(result_type)) + return nullptr; + auto extract_kv_from_map = [](const ColumnMap * map) { const ColumnTuple & tuple = map->getNestedData(); diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference new file mode 100644 index 00000000000..7dabd5388f4 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference @@ -0,0 +1,10 @@ +('0') Variant(Tuple(String), Tuple(\n number UInt64)) +(1) Variant(Tuple(String), Tuple(\n number UInt64)) +('2') Variant(Tuple(String), Tuple(\n number UInt64)) +(3) Variant(Tuple(String), Tuple(\n number UInt64)) +('4') Variant(Tuple(String), Tuple(\n number UInt64)) +{'0':'0'} Variant(Map(String, String), Map(UInt64, UInt64)) +{1:1} Variant(Map(String, String), Map(UInt64, UInt64)) +{'2':'2'} Variant(Map(String, String), Map(UInt64, UInt64)) +{3:3} Variant(Map(String, String), Map(UInt64, UInt64)) +{'4':'4'} Variant(Map(String, String), Map(UInt64, UInt64)) diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql new file mode 100644 index 00000000000..4a9a788ab18 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql @@ -0,0 +1,7 @@ +set use_variant_as_common_type = 1; +set allow_experimental_variant_type = 1; + +SELECT if(number % 2, tuple(number), tuple(toString(number))) as res, toTypeName(res) FROM numbers(5); +SELECT if(number % 2, map(number, number), map(toString(number), toString(number))) as res, toTypeName(res) FROM numbers(5); + + From b38c46a87d6eacbc7805562deb07ce586fd7e0fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 415/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From e3239c6ee11eb5bf0466fb750c58125868885ec8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 19:09:16 +0200 Subject: [PATCH 416/496] Fix bad log message in JIT for sorting --- src/Core/SortDescription.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 9edc79a1ff1..1b3f81f8547 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -103,7 +103,15 @@ static std::string getSortDescriptionDump(const SortDescription & description, c WriteBufferFromOwnString buffer; for (size_t i = 0; i < description.size(); ++i) - buffer << header_types[i]->getName() << ' ' << description[i].direction << ' ' << description[i].nulls_direction; + { + if (i != 0) + buffer << ", "; + + buffer << "(type: " << header_types[i]->getName() + << ", direction: " << description[i].direction + << ", nulls_direction: " << description[i].nulls_direction + << ")"; + } return buffer.str(); } From 3c4389ec4d78db55ce742e5d5a3b0ed050c9c9e6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 2 Aug 2024 14:57:19 -0300 Subject: [PATCH 417/496] doc/fix max_partitions_to_read description --- .../operations/settings/merge-tree-settings.md | 7 ++----- .../en/operations/settings/query-complexity.md | 18 ++++++++++++++++-- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 7278b91f90d..67fa45c20cd 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -119,11 +119,6 @@ Minimum size of blocks of uncompressed data required for compression when writin You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. -## max_partitions_to_read - -Limits the maximum number of partitions that can be accessed in one query. -You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. - ## max_suspicious_broken_parts If the number of broken parts in a single partition exceeds the `max_suspicious_broken_parts` value, automatic deletion is denied. @@ -691,6 +686,8 @@ Possible values: Default value: -1 (unlimited). +You can also specify a query complexity setting [max_partitions_to_read](query-complexity#max-partitions-to-read) at a query / session / profile level. + ## min_age_to_force_merge_seconds {#min_age_to_force_merge_seconds} Merge parts if every part in the range is older than the value of `min_age_to_force_merge_seconds`. diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 2a20e74e20f..14ccb1167f9 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -188,7 +188,7 @@ If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use c What to do if the query is run longer than `max_execution_time` or the estimated running time is longer than `max_estimated_execution_time`: `throw` or `break`. By default, `throw`. -# max_execution_time_leaf +## max_execution_time_leaf Similar semantic to `max_execution_time` but only apply on leaf node for distributed or remote queries. @@ -204,7 +204,7 @@ We can use `max_execution_time_leaf` as the query settings: SELECT count() FROM cluster(cluster, view(SELECT * FROM t)) SETTINGS max_execution_time_leaf = 10; ``` -# timeout_overflow_mode_leaf +## timeout_overflow_mode_leaf What to do when the query in leaf node run longer than `max_execution_time_leaf`: `throw` or `break`. By default, `throw`. @@ -426,3 +426,17 @@ Example: ``` Default value: 0 (Infinite count of simultaneous sessions). + +## max_partitions_to_read {#max-partitions-to-read} + +Limits the maximum number of partitions that can be accessed in one query. + +The setting value specified when the table is created can be overridden via query-level setting. + +Possible values: + +- Any positive integer. + +Default value: -1 (unlimited). + +You can also specify a MergeTree setting [max_partitions_to_read](merge-tree-settings#max-partitions-to-read) in tables' setting. From ce39957983af8bdd7d97e4a3729e2f97d3e0cb85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:11:25 +0200 Subject: [PATCH 418/496] Remove capitalization in test reports --- tests/ci/report.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index f50ed4c1f85..3f0fc596824 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -293,9 +293,9 @@ class JobReport: start_time: str duration: float additional_files: Union[Sequence[str], Sequence[Path]] - # clickhouse version, build job only + # ClickHouse version, build job only version: str = "" - # checkname to set in commit status, set if differs from jjob name + # check_name to be set in commit status, set it if it differs from the job name check_name: str = "" # directory with artifacts to upload on s3 build_dir_for_upload: Union[Path, str] = "" @@ -667,11 +667,7 @@ ColorTheme = Tuple[str, str, str] def _format_header( header: str, branch_name: str, branch_url: Optional[str] = None ) -> str: - # Following line does not lower CI->Ci and SQLancer->Sqlancer. It only - # capitalizes the first letter and doesn't touch the rest of the word - result = " ".join([w[0].upper() + w[1:] for w in header.split(" ") if w]) - result = result.replace("Clickhouse", "ClickHouse") - result = result.replace("clickhouse", "ClickHouse") + result = header if "ClickHouse" not in result: result = f"ClickHouse {result}" if branch_url: From 2c9b61d047c1afe22b0fa0a967a87db8bd4cf62f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:16:44 +0200 Subject: [PATCH 419/496] Miscellaneous --- tests/ci/ci.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 935fe472e50..6ca84a346e2 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1019,7 +1019,7 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: +def _cancel_pr_workflow(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: wf_data = CiMetadata(s3, pr_number).fetch_meta() if not cancel_sync: if not wf_data.run_id: @@ -1368,12 +1368,12 @@ def main() -> int: assert indata, "Run config must be provided via --infile" _update_gh_statuses_action(indata=indata, s3=s3) - ### CANCEL PREVIOUS WORKFLOW RUN + ### CANCEL THE PREVIOUS WORKFLOW RUN elif args.cancel_previous_run: if pr_info.is_merge_queue: - _cancel_pr_wf(s3, pr_info.merged_pr) + _cancel_pr_workflow(s3, pr_info.merged_pr) elif pr_info.is_pr: - _cancel_pr_wf(s3, pr_info.number, cancel_sync=True) + _cancel_pr_workflow(s3, pr_info.number, cancel_sync=True) else: assert False, "BUG! Not supported scenario" From bd3606dac4954c673ec6c38dd6fbdb70bc7b53cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:17:07 +0200 Subject: [PATCH 420/496] Fix typos --- tests/ci/commit_status_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index fdc9c002b66..908ac4a7dca 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -301,7 +301,7 @@ def get_worst_state(statuses: CommitStatuses) -> StatusType: def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: - """The function converst the statuses to TestResults and uploads the report + """The function converts the statuses to TestResults and uploads the report to S3 tests bucket. Then it returns the URL""" test_results = [] # type: TestResults for status in statuses: From 675afda17210ca7e8e71e0899a5ed14d7227fb55 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:22:08 +0200 Subject: [PATCH 421/496] Fix check names in the CI Logs database --- tests/ci/clickhouse_helper.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 287970cce9a..0725f7100d1 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -3,6 +3,7 @@ import fileinput import json import logging import time +import os from pathlib import Path from typing import Any, Dict, List, Optional @@ -298,6 +299,11 @@ class CiLogsCredentials: def get_docker_arguments( self, pr_info: PRInfo, check_start_time: str, check_name: str ) -> str: + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) + if run_by_hash_total > 1: + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + check_name = f"{check_name} [{run_by_hash_num + 1}/{run_by_hash_total}]" + self.create_ci_logs_credentials() if not self.config_path.exists(): logging.info("Do not use external logs pushing") From b282be83c500bac5544424378b9505fc8c28e432 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 20:03:43 +0200 Subject: [PATCH 422/496] remove old workflows --- .github/actions/release/action.yml | 166 --------------------------- .github/workflows/auto_release.yml | 111 ------------------ .github/workflows/create_release.yml | 2 + .github/workflows/release.yml | 71 ------------ .github/workflows/tags_stable.yml | 74 ------------ tests/ci/docker_server.py | 1 - 6 files changed, 2 insertions(+), 423 deletions(-) delete mode 100644 .github/actions/release/action.yml delete mode 100644 .github/workflows/auto_release.yml delete mode 100644 .github/workflows/release.yml delete mode 100644 .github/workflows/tags_stable.yml diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml deleted file mode 100644 index a287aa8b41d..00000000000 --- a/.github/actions/release/action.yml +++ /dev/null @@ -1,166 +0,0 @@ -name: Release - -description: Makes patch releases and creates new release branch - -inputs: - ref: - description: 'Git reference (branch or commit sha) from which to create the release' - required: true - type: string - type: - description: 'The type of release: "new" for a new release or "patch" for a patch release' - required: true - type: choice - options: - - patch - - new - dry-run: - description: 'Dry run' - required: true - type: boolean - token: - required: true - type: string - -runs: - using: "composite" - steps: - - name: Prepare Release Info - shell: bash - run: | - python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} - echo "::group::Release Info" - python3 -m json.tool /tmp/release_info.json - echo "::endgroup::" - release_tag=$(jq -r '.release_tag' /tmp/release_info.json) - commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) - echo "Release Tag: $release_tag" - echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" - echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" - - name: Download All Release Artifacts - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Push Git Tag for the Release - shell: bash - run: | - python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - shell: bash - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - git checkout master - python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" - echo "List versions" - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - echo "Update docker version" - ./utils/list-versions/update-docker-version.sh - echo "Generate ChangeLog" - export CI=1 - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token=${{ inputs.token }} --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} - git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md - echo "Generate Security" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} - uses: peter-evans/create-pull-request@v6 - with: - author: "robot-clickhouse " - token: ${{ inputs.token }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - branch: auto/${{ env.RELEASE_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) - - name: Complete previous steps and Restore git state - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --set-progress-completed - git reset --hard HEAD - git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - shell: bash - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export TGZ Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test TGZ Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export RPM Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test RPM Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export Debian Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test Debian Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Docker clickhouse/clickhouse-server building - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - cd "./tests/ci" - python3 ./create_release.py --set-progress-started --progress "docker server release" - export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./create_release.py --set-progress-completed - - name: Docker clickhouse/clickhouse-keeper building - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - cd "./tests/ci" - python3 ./create_release.py --set-progress-started --progress "docker keeper release" - export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./create_release.py --set-progress-completed - - name: Set current Release progress to Completed with OK - shell: bash - run: | - python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" - python3 ./tests/ci/create_release.py --set-progress-completed - - name: Post Slack Message - if: ${{ !cancelled() }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml deleted file mode 100644 index 457ffacc7a8..00000000000 --- a/.github/workflows/auto_release.yml +++ /dev/null @@ -1,111 +0,0 @@ -name: AutoRelease - -env: - PYTHONUNBUFFERED: 1 - DRY_RUN: true - -concurrency: - group: release -on: # yamllint disable-line rule:truthy - # Workflow uses a test bucket for packages and dry run mode (no real releases) - schedule: - - cron: '0 9 * * *' - - cron: '0 15 * * *' - workflow_dispatch: - inputs: - dry-run: - description: 'Dry run' - required: false - default: true - type: boolean - -jobs: - AutoRelease: - runs-on: [self-hosted, release-maker] - steps: - - name: DebugInfo - uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" - - name: Set DRY_RUN for dispatch - if: ${{ github.event_name == 'workflow_dispatch' }} - run: echo "DRY_RUN=${{ github.event.inputs.dry-run }}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - fetch-depth: 0 - - name: Auto Release Prepare - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --prepare - echo "::group::Auto Release Info" - python3 -m json.tool /tmp/autorelease_info.json - echo "::endgroup::" - { - echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_ENV" - - name: Post Release Branch statuses - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --post-status - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[0].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Post Slack Message - if: ${{ !cancelled() }} - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} - - name: Clean up - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 29094cc51a6..d4993b373df 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -89,6 +89,8 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" + + git checkout master # in case WF started from feature branch echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv echo "Update docker version" diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml deleted file mode 100644 index 7dc4e3298a6..00000000000 --- a/.github/workflows/release.yml +++ /dev/null @@ -1,71 +0,0 @@ -name: PublishedReleaseCI -# - Gets artifacts from S3 -# - Sends it to JFROG Artifactory -# - Adds them to the release assets - -on: # yamllint disable-line rule:truthy - release: - types: - - published - workflow_dispatch: - inputs: - tag: - description: 'Release tag' - required: true - type: string - -jobs: - ReleasePublish: - runs-on: [self-hosted, style-checker] - steps: - - name: Set tag from input - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Set tag from REF - if: github.event_name == 'release' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Deploy packages and assets - run: | - curl --silent --data '' --no-buffer \ - '${{ secrets.PACKAGES_RELEASE_URL }}/release/'"${GITHUB_TAG}"'?binary=binary_darwin&binary=binary_darwin_aarch64&sync=true' - ############################################################################################ - ##################################### Docker images ####################################### - ############################################################################################ - DockerServerImages: - runs-on: [self-hosted, style-checker] - steps: - - name: Set tag from input - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Set tag from REF - if: github.event_name == 'release' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - fetch-depth: 0 # otherwise we will have no version info - filter: tree:0 - ref: ${{ env.GITHUB_TAG }} - - name: Check docker clickhouse/clickhouse-server building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - export CHECK_NAME="Docker server image" - SHA=$(git rev-list -n 1 "$GITHUB_TAG") - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - - name: Check docker clickhouse/clickhouse-keeper building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - export CHECK_NAME="Docker keeper image" - SHA=$(git rev-list -n 1 "$GITHUB_TAG") - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml deleted file mode 100644 index 2aa7694bc41..00000000000 --- a/.github/workflows/tags_stable.yml +++ /dev/null @@ -1,74 +0,0 @@ -name: TagsStableWorkflow -# - Gets artifacts from S3 -# - Sends it to JFROG Artifactory -# - Adds them to the release assets - -env: - # Force the stdout and stderr streams to be unbuffered - PYTHONUNBUFFERED: 1 - -on: # yamllint disable-line rule:truthy - push: - tags: - - 'v*-prestable' - - 'v*-stable' - - 'v*-lts' - workflow_dispatch: - inputs: - tag: - description: 'Test tag' - required: true - type: string - - -jobs: - UpdateVersions: - runs-on: [self-hosted, style-checker] - steps: - - name: Set test tag - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Get tag name - if: github.event_name != 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - ref: master - fetch-depth: 0 - filter: tree:0 - - name: Update versions, docker version, changelog, security - env: - GITHUB_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - run: | - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - ./utils/list-versions/update-docker-version.sh - GID=$(id -g "${UID}") - # --network=host and CI=1 are required for the S3 access from a container - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume="${GITHUB_WORKSPACE}:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token="$GITHUB_TOKEN" --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" "${GITHUB_TAG}" - git add "./docs/changelogs/${GITHUB_TAG}.md" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create Pull Request - uses: peter-evans/create-pull-request@v6 - with: - author: "robot-clickhouse " - token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - branch: auto/${{ env.GITHUB_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 8f0474d5053..3251ec5644e 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -27,7 +27,6 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from version_helper import ( ClickHouseVersion, - get_tagged_versions, get_version_from_repo, version_arg, ) From aa38e78d7238d843737d1d268de6ee189c19edc3 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 20:27:59 +0200 Subject: [PATCH 423/496] update version_date.tsv --- docs/changelogs/v23.8.16.40-lts.md | 35 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 36 insertions(+) create mode 100644 docs/changelogs/v23.8.16.40-lts.md diff --git a/docs/changelogs/v23.8.16.40-lts.md b/docs/changelogs/v23.8.16.40-lts.md new file mode 100644 index 00000000000..75caf1ea277 --- /dev/null +++ b/docs/changelogs/v23.8.16.40-lts.md @@ -0,0 +1,35 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.16.40-lts (e143a9039ba) FIXME as compared to v23.8.15.35-lts (060ff8e813a) + +#### Improvement +* Backported in [#66962](https://github.com/ClickHouse/ClickHouse/issues/66962): Added support for parameterized view with analyzer to not analyze create parameterized view. Refactor existing parameterized view logic to not analyze create parameterized view. [#54211](https://github.com/ClickHouse/ClickHouse/pull/54211) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Backported in [#65461](https://github.com/ClickHouse/ClickHouse/issues/65461): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Backported in [#65880](https://github.com/ClickHouse/ClickHouse/issues/65880): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65912](https://github.com/ClickHouse/ClickHouse/issues/65912): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65281](https://github.com/ClickHouse/ClickHouse/issues/65281): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65368](https://github.com/ClickHouse/ClickHouse/issues/65368): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#65743](https://github.com/ClickHouse/ClickHouse/issues/65743): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65351](https://github.com/ClickHouse/ClickHouse/issues/65351): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66037](https://github.com/ClickHouse/ClickHouse/issues/66037): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65782](https://github.com/ClickHouse/ClickHouse/issues/65782): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65926](https://github.com/ClickHouse/ClickHouse/issues/65926): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65822](https://github.com/ClickHouse/ClickHouse/issues/65822): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66449](https://github.com/ClickHouse/ClickHouse/issues/66449): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66717](https://github.com/ClickHouse/ClickHouse/issues/66717): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65080](https://github.com/ClickHouse/ClickHouse/issues/65080): Follow up to [#56541](https://github.com/ClickHouse/ClickHouse/issues/56541). [#57141](https://github.com/ClickHouse/ClickHouse/pull/57141) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#65913](https://github.com/ClickHouse/ClickHouse/issues/65913): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66853](https://github.com/ClickHouse/ClickHouse/issues/66853): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 24488066190..cb6b8f588da 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -50,6 +50,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.16.40-lts 2024-08-02 v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 From a45ba44dbaa2ed43eb63e49fe609a01be978eac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 18:28:38 +0000 Subject: [PATCH 424/496] Automatic style fix --- tests/ci/ci.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 6ca84a346e2..805296d2bb2 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1019,7 +1019,9 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def _cancel_pr_workflow(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: +def _cancel_pr_workflow( + s3: S3Helper, pr_number: int, cancel_sync: bool = False +) -> None: wf_data = CiMetadata(s3, pr_number).fetch_meta() if not cancel_sync: if not wf_data.run_id: From a37eeb0f211c0c7b6251a9108d6b939e73c9a66e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Aug 2024 20:13:27 +0000 Subject: [PATCH 425/496] Allow types to change, re-resolve overloads --- src/Storages/MergeTree/KeyCondition.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index eaf9f0af623..91f054c3a71 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -627,7 +627,8 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - res = &inverted_dag.addFunction(node.function_base, children, ""); + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); handled_inversion = true; } else if (need_inversion && (name == "and" || name == "or")) @@ -668,8 +669,13 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - res = &inverted_dag.addFunction(node.function_base, children, ""); - chassert(res->result_type == node.result_type); + /// Can't just addFunction(node.function_base) because argument types may have + /// changed slightly because of our transformations, e.g. maybe some subexpression + /// changed constness, which caused some function return value to change LowCardinality-ness. + /// (I don't have a specific counterexample, but it seems likely that it exists. + /// One was fixed in the past: https://github.com/ClickHouse/ClickHouse/issues/65143 ) + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); } } } @@ -678,13 +684,6 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( if (!handled_inversion && need_inversion) res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, ""); - /// Make sure we don't change any data types (e.g. remove LowCardinality). - /// If it turns out that we actually want to change data types sometimes, it's ok to remove this - /// check *and* replace all `addFunction(node.function_base, ...)` calls above with - /// `addFunction(FunctionFactory::instance().get(name, context), ...)` to re-resolve overloads. - if (!node.result_type->equals(*res->result_type)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", node.result_type->getName(), res->result_type->getName(), node.result_name); - to_inverted[&node] = res; return *res; } From eb0e12099a8fa45d79b2cff96e02ee273e879efa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Aug 2024 21:18:35 +0000 Subject: [PATCH 426/496] Another attempt --- src/Storages/MergeTree/KeyCondition.cpp | 27 ++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 91f054c3a71..9115cb5608e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -627,8 +627,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - auto function_builder = FunctionFactory::instance().get(name, context); - res = &inverted_dag.addFunction(function_builder, children, ""); + res = &inverted_dag.addFunction(node.function_base, children, ""); handled_inversion = true; } else if (need_inversion && (name == "and" || name == "or")) @@ -669,13 +668,23 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - /// Can't just addFunction(node.function_base) because argument types may have - /// changed slightly because of our transformations, e.g. maybe some subexpression - /// changed constness, which caused some function return value to change LowCardinality-ness. - /// (I don't have a specific counterexample, but it seems likely that it exists. - /// One was fixed in the past: https://github.com/ClickHouse/ClickHouse/issues/65143 ) - auto function_builder = FunctionFactory::instance().get(name, context); - res = &inverted_dag.addFunction(function_builder, children, ""); + /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). + /// Otherwise the function may crash when passed columns of unexpected types. + /// * Why not check this for all subexperessions rather than function arguments? + /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` + /// the AND's args were UInt64, but OR's args are UInt8. + /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? + /// Because some functions can't be found through FunctionFactory, e.g. FunctionCapture. + /// (But maybe we could re-resolve only if argument types changed.) + for (size_t i = 0; i < children.size(); ++i) + { + if (!node.children[i]->result_type->equals(*children[i]->result_type)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", + node.children[i]->result_type->getName(), children[i]->result_type->getName(), node.children[i]->result_name); + } + + res = &inverted_dag.addFunction(node.function_base, children, ""); } } } From dd0ae04f90314ce6d5dbe748605e66f1a6d9024f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 427/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From a431ab3e4b6f925924a81d99997e6c028ae7950f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:31:02 +0200 Subject: [PATCH 428/496] Improve dashboard --- programs/server/dashboard.html | 66 +++++++++++++++++++++++++++------- 1 file changed, 54 insertions(+), 12 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 45f988f7b1e..71880b9e228 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -17,7 +17,7 @@ --input-shadow-color: rgba(0, 255, 0, 1); --error-color: red; --global-error-color: white; - --legend-background: rgba(255, 255, 255, 0.75); + --legend-background: rgba(255, 255, 0, 0.75); --title-color: #666; --text-color: black; --edit-title-background: #FEE; @@ -41,7 +41,7 @@ --moving-shadow-color: rgba(255, 255, 255, 0.25); --input-shadow-color: rgba(255, 128, 0, 0.25); --error-color: #F66; - --legend-background: rgba(255, 255, 255, 0.25); + --legend-background: rgba(0, 96, 128, 0.75); --title-color: white; --text-color: white; --edit-title-background: #364f69; @@ -1004,14 +1004,14 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- className && legendEl.classList.add(className); uPlot.assign(legendEl.style, { - textAlign: "left", + textAlign: "right", pointerEvents: "none", display: "none", position: "absolute", left: 0, top: 0, - zIndex: 100, - boxShadow: "2px 2px 10px rgba(0,0,0,0.1)", + zIndex: 200, + boxShadow: "2px 2px 10px rgba(0, 0, 0, 0.1)", ...style }); @@ -1051,8 +1051,10 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- function update(u) { let { left, top } = u.cursor; - left -= legendEl.clientWidth / 2; - top -= legendEl.clientHeight / 2; + /// This will make the balloon to the right of the cursor when the cursor is on the left side, and vise-versa, + /// avoiding the borders of the chart. + left -= legendEl.clientWidth * (left / u.width); + top -= legendEl.clientHeight; legendEl.style.transform = "translate(" + left + "px, " + top + "px)"; if (multiline) { @@ -1229,14 +1231,53 @@ async function draw(idx, chart, url_params, query) { let sync = uPlot.sync("sync"); - let axis = { + function formatDateTime(t) { + return (new Date(t * 1000)).toISOString().replace('T', '\n').replace('.000Z', ''); + } + + function formatDateTimes(self, ticks) { + return ticks.map((t, idx) => { + let res = formatDateTime(t); + if (idx == 0 || res.substring(0, 10) != formatDateTime(ticks[idx - 1]).substring(0, 10)) { + return res; + } else { + return res.substring(11); + } + }); + } + + function formatValue(v) { + const a = Math.abs(v); + if (a >= 1000000000000000) { return (v / 1000000000000000) + 'P'; } + if (a >= 1000000000000) { return (v / 1000000000000) + 'T'; } + if (a >= 1000000000) { return (v / 1000000000) + 'G'; } + if (a >= 1000000) { return (v / 1000000) + 'M'; } + if (a >= 1000) { return (v / 1000) + 'K'; } + if (a > 0 && a < 0.001) { return (v * 1000000) + "μ"; } + return v; + } + + let axis_x = { stroke: axes_color, grid: { width: 1 / devicePixelRatio, stroke: grid_color }, - ticks: { width: 1 / devicePixelRatio, stroke: grid_color } + ticks: { width: 1 / devicePixelRatio, stroke: grid_color }, + values: formatDateTimes, + space: 80, + incrs: [1, 5, 10, 15, 30, + 60, 60 * 5, 60 * 10, 60 * 15, 60 * 30, + 3600, 3600 * 2, 3600 * 3, 3600 * 4, 3600 * 6, 3600 * 12, + 3600 * 24], }; - let axes = [axis, axis]; - let series = [{ label: "x" }]; + let axis_y = { + stroke: axes_color, + grid: { width: 1 / devicePixelRatio, stroke: grid_color }, + ticks: { width: 1 / devicePixelRatio, stroke: grid_color }, + values: (self, ticks) => ticks.map(formatValue) + }; + + let axes = [axis_x, axis_y]; + let series = [{ label: "time", value: (self, t) => formatDateTime(t) }]; let data = [reply.data[reply.meta[0].name]]; // Treat every column as series @@ -1254,9 +1295,10 @@ async function draw(idx, chart, url_params, query) { const opts = { width: chart.clientWidth, height: chart.clientHeight, + scales: { x: { time: false } }, /// Because we want to split and format time on our own. axes, series, - padding: [ null, null, null, (Math.round(max_value * 100) / 100).toString().length * 6 - 10 ], + padding: [ null, null, null, 3 ], plugins: [ legendAsTooltipPlugin() ], cursor: { sync: { From a6f9dd4447cbb475cbf77b07de35b40fbcad50b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:35:10 +0200 Subject: [PATCH 429/496] Improve dashboard --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 71880b9e228..c69acec7858 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1010,7 +1010,7 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- position: "absolute", left: 0, top: 0, - zIndex: 200, + zIndex: 100, boxShadow: "2px 2px 10px rgba(0, 0, 0, 0.1)", ...style }); From 95659de26573bdb17ab2b5649e6dad96fb75c479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:45:41 +0200 Subject: [PATCH 430/496] Fix invalid detection of an empty result --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index c69acec7858..238254f4ef8 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1141,7 +1141,7 @@ async function draw(idx, chart, url_params, query) { let {reply, error} = await doFetch(query, url_params); if (!error) { - if (reply.rows.length == 0) { + if (reply.rows == 0) { error = "Query returned empty result."; } else if (reply.meta.length < 2) { error = "Query should return at least two columns: unix timestamp and value."; From a99f9bb603f78437fba8d3ebb031c2f41d00cd58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:50:27 +0200 Subject: [PATCH 431/496] Focus on the mass editor --- programs/server/dashboard.html | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 238254f4ef8..8fb07d5da3b 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -945,6 +945,7 @@ function showMassEditor() { let editor = document.getElementById('mass-editor-textarea'); editor.value = JSON.stringify({params: params, queries: queries}, null, 2); + editor.focus(); mass_editor_active = true; } From eeb8c1caac9e8e2ba2f3a1a86f5603281e161610 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 02:06:53 +0200 Subject: [PATCH 432/496] Improve margins when there are many parameters --- programs/server/dashboard.html | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 8fb07d5da3b..344de779065 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -256,6 +256,7 @@ font-weight: bold; user-select: none; cursor: pointer; + margin-bottom: 1rem; } #run:hover { @@ -309,7 +310,7 @@ color: var(--param-text-color); display: inline-block; box-shadow: 1px 1px 0 var(--shadow-color); - margin-bottom: 1rem; + margin-bottom: 0.5rem; } input:focus { From 090fb59194462324507d75f032aa803303c3e041 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 02:20:58 +0200 Subject: [PATCH 433/496] Automatic field width of chart parameters --- programs/server/dashboard.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 344de779065..0b099b15536 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -218,6 +218,7 @@ #chart-params .param { width: 6%; + font-family: monospace; } input { @@ -658,6 +659,10 @@ function insertParam(name, value) { param_value.value = value; param_value.spellcheck = false; + let setWidth = e => { e.style.width = (e.value.length + 1) + 'ch' }; + if (value) { setWidth(param_value); } + param_value.addEventListener('input', e => setWidth(e.target)); + param_wrapper.appendChild(param_name); param_wrapper.appendChild(param_value); document.getElementById('chart-params').appendChild(param_wrapper); From 9a017528a4685fc4ed7eec7ba37f9e9804972c3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 434/496] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From dfeb1991164bd6c8b0efc8bdcfe9dcd5b8906928 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 03:06:37 +0200 Subject: [PATCH 435/496] Fix locking inside TimerDescriptor --- src/Common/TimerDescriptor.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 9a171ae9487..ce290a1cb31 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -75,10 +76,22 @@ void TimerDescriptor::drain() const /// or since the last successful read(2), then the buffer given to read(2) returns an unsigned 8-byte integer (uint64_t) /// containing the number of expirations that have occurred. /// (The returned value is in host byte order—that is, the native byte order for integers on the host machine.) + + /// Due to a bug in Linux Kernel, reading from timerfd in non-blocking mode can be still blocking. + /// Avoid it with polling. + Epoll epoll; + epoll.add(timer_fd); + epoll_event event; + event.data.fd = -1; + size_t ready_count = epoll.getManyReady(1, &event, 0); + if (!ready_count) + return; + uint64_t buf; while (true) { ssize_t res = ::read(timer_fd, &buf, sizeof(buf)); + if (res < 0) { /// man timerfd_create: From e491c51cdc9a94018fc6918f74232cfff747436a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 3 Aug 2024 01:49:53 +0000 Subject: [PATCH 436/496] e --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9115cb5608e..2b89344d3d9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -670,7 +670,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( { /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). /// Otherwise the function may crash when passed columns of unexpected types. - /// * Why not check this for all subexperessions rather than function arguments? + /// * Why not check this for all subexpressions rather than function arguments? /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` /// the AND's args were UInt64, but OR's args are UInt8. /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? From 8cb2e308f41638ebb6ba7fddbd4f0bf89d4d612e Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Sat, 3 Aug 2024 01:55:24 +0000 Subject: [PATCH 437/496] Only new analyzer --- .../03209_parameterized_view_with_non_literal_params.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql index f2c61e5cb1d..7fe84929910 100644 --- a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -1,4 +1,4 @@ - +SET allow_experimental_analyzer = 1; select 'Test with Date parameter'; drop table if exists date_table_pv; From 96e826d154e9b55cf035a3ce025ac81194455ebc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 3 Aug 2024 02:57:33 +0000 Subject: [PATCH 438/496] Change tactics again --- src/Storages/MergeTree/KeyCondition.cpp | 31 +++++++++++++++---------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2b89344d3d9..dfb43c4e75d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -668,23 +668,30 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). - /// Otherwise the function may crash when passed columns of unexpected types. - /// * Why not check this for all subexpressions rather than function arguments? - /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` - /// the AND's args were UInt64, but OR's args are UInt8. - /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? - /// Because some functions can't be found through FunctionFactory, e.g. FunctionCapture. - /// (But maybe we could re-resolve only if argument types changed.) + /// Argument types could change slightly because of our transformations, e.g. + /// LowCardinality can be added because some subexpressions became constant + /// (in particular, sets). If that happens, re-run function overload resolver. + /// Otherwise don't re-run it because some functions may not be available + /// through FunctionFactory::get(), e.g. FunctionCapture. + bool types_changed = false; for (size_t i = 0; i < children.size(); ++i) { if (!node.children[i]->result_type->equals(*children[i]->result_type)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", - node.children[i]->result_type->getName(), children[i]->result_type->getName(), node.children[i]->result_name); + { + types_changed = true; + break; + } } - res = &inverted_dag.addFunction(node.function_base, children, ""); + if (types_changed) + { + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); + } + else + { + res = &inverted_dag.addFunction(node.function_base, children, ""); + } } } } From f97abf69949f8822d70f4b1251e1945f279dd0ec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 09:35:37 +0200 Subject: [PATCH 439/496] tests: avoid endless wait in 01042_system_reload_dictionary_reloads_completely Signed-off-by: Azat Khuzhin --- ...em_reload_dictionary_reloads_completely.sh | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index 03dd376f802..ebc4110332f 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -8,6 +8,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail +# Wait when the dictionary will update the value for 13 on its own: +function wait_for_dict_upate() +{ + for ((i = 0; i < 100; ++i)); do + if [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))")" != -1 ]; then + return 0 + fi + sleep 0.5 + done + return 1 +} + $CLICKHOUSE_CLIENT < ', dictGetInt64('${CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (13, 103, now())" $CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (14, 104, now() - INTERVAL 1 DAY)" -# Wait when the dictionary will update the value for 13 on its own: -while [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))")" = -1 ] -do - sleep 0.5 -done +if ! wait_for_dict_upate; then + echo "Dictionary had not been reloaded" >&2 + exit 1 +fi $CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" From 40cd5467c18d65a6624d273ac1a8fd9cc9257d8c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 09:39:33 +0200 Subject: [PATCH 440/496] tests: fix 01042_system_reload_dictionary_reloads_completely flakiness (increase lag) The test fails in case of INSERT takes > 1 sec: 2024.08.02 13:06:07.746869 [ 45445 ] {c9b55378-6bc5-46d5-80c1-5385a880f88b} executeQuery: (from [::1]:37208) (comment: 01042_system_reload_dictionary_reloads_completely.sh) CREATE DICTIONARY test_m4lx2bit.dict ( x Int64 DEFAULT -1, y Int64 DEFAULT -1, insert_time DateTime ) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB 'test_m4lx2bit' UPDATE_FIELD 'insert_time')) LAYOUT(FLAT()) LIFETIME(1); (stage: Complete) ... 2024.08.02 13:06:08.263832 [ 59987 ] {744da223-67b9-4e32-b12a-eb2550a92fdb} DictionaryFactory: Created dictionary source 'ClickHouse: test_m4lx2bit.table' for dictionary '5b2b98a9-9372-47c9-bda3-830794cb96e7' 2024.08.02 13:06:08.268118 [ 59987 ] {744da223-67b9-4e32-b12a-eb2550a92fdb} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table`; (stage: Complete) ... 2024.08.02 13:06:09.193190 [ 45445 ] {b6033498-4666-452f-bcf9-02ecf257ba7f} executeQuery: (from [::1]:37262) (comment: 01042_system_reload_dictionary_reloads_completely.sh) INSERT INTO test_m4lx2bit.table VALUES (stage: Complete) ... 2024.08.02 13:06:11.342119 [ 50962 ] {} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table` WHERE insert_time >= '2024-08-02 13:06:07'; (stage: Complete) ... 2024.08.02 13:06:11.832158 [ 45445 ] {b6033498-4666-452f-bcf9-02ecf257ba7f} TCPHandler: Processed in 2.642106236 sec. ... 2024.08.02 13:06:16.357448 [ 41632 ] {} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table` WHERE insert_time >= '2024-08-02 13:06:10'; (stage: Complete) Signed-off-by: Azat Khuzhin --- .../01042_system_reload_dictionary_reloads_completely.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index ebc4110332f..453e1bb8f0a 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -31,7 +31,7 @@ CREATE DICTIONARY ${CLICKHOUSE_DATABASE}.dict insert_time DateTime ) PRIMARY KEY x -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB '${CLICKHOUSE_DATABASE}' UPDATE_FIELD 'insert_time')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB '${CLICKHOUSE_DATABASE}' UPDATE_FIELD 'insert_time' UPDATE_LAG 60)) LAYOUT(FLAT()) LIFETIME(1); EOF From 6ce6af0647590f4b58a6ab87ee5f29b8487e8c2f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 3 Aug 2024 14:16:24 +0200 Subject: [PATCH 441/496] Fix completion RESTORE ON CLUSTER. --- src/Backups/RestorerFromBackup.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 3056f9fe421..278af9d4eb3 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -222,10 +222,19 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa if (restore_coordination) { restore_coordination->setStage(new_stage, message); - if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) - restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); - else - restore_coordination->waitForStage(new_stage); + + /// The initiator of a RESTORE ON CLUSTER query waits for other hosts to complete their work (see waitForStage(Stage::COMPLETED) in BackupsWorker::doRestore), + /// but other hosts shouldn't wait for each others' completion. (That's simply unnecessary and also + /// the initiator may start cleaning up (e.g. removing restore-coordination ZooKeeper nodes) once all other hosts are in Stage::COMPLETED.) + bool need_wait = (new_stage != Stage::COMPLETED); + + if (need_wait) + { + if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) + restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); + else + restore_coordination->waitForStage(new_stage); + } } } From 73080d25a93aa715c4126ed82b89e3ada89d7a3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 15:10:05 +0200 Subject: [PATCH 442/496] Fix test `00002_log_and_exception_messages_formatting` --- src/Databases/DatabaseHDFS.cpp | 4 ++-- src/Storages/ObjectStorage/HDFS/Configuration.cpp | 6 +++--- .../00002_log_and_exception_messages_formatting.sql | 5 ++++- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index eccaae5f22e..f58f1b76e71 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -75,8 +75,8 @@ std::string DatabaseHDFS::getTablePath(const std::string & table_name) const return table_name; if (source.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. " - "It should have structure 'hdfs://:/path'", table_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. " + "It should have the following structure 'hdfs://:/path'", table_name); return fs::path(source) / table_name; } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index e8071be6f02..85eb29a3868 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -142,11 +142,11 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) { auto pos = url_.find("//"); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have the following structure 'hdfs://:/path'", url_); pos = url_.find('/', pos + 2); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have the following structure 'hdfs://:/path'", url_); path = url_.substr(pos + 1); if (!path.starts_with('/')) @@ -155,7 +155,7 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) url = url_.substr(0, pos); paths = {path}; - LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); + LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using URL: {}, path: {}", url, path); } void StorageHDFSConfiguration::addStructureAndFormatToArgs( diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 07c42d6d039..e916fdfc1ff 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -163,7 +163,10 @@ create temporary table known_short_messages (s String) as select * from (select '{} -> {}', '{} {}', '{}%', - '{}: {}' + '{}: {}', + 'Unknown data type family: {}', + 'Cannot load time zone {}', + 'Unknown table engine {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From a749223251b7e580f5d7bbcb4fc59aa6b5fffbe2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 3 Aug 2024 22:43:12 +0800 Subject: [PATCH 443/496] change as request --- src/Functions/printf.cpp | 39 ++++++++++++++----- .../0_stateless/03203_function_printf.sql | 7 +++- 2 files changed, 36 insertions(+), 10 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index 3efe854a53b..3cf3efaf534 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -6,11 +6,10 @@ #include #include #include +#include #include #include -#include -#include #include #include #include @@ -22,6 +21,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int BAD_ARGUMENTS; } namespace @@ -52,10 +52,9 @@ private: [[maybe_unused]] String toString() const { - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() - << std::endl; - return oss.str(); + WriteBufferFromOwnString buf; + buf << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() << "\n"; + return buf.str(); } private: @@ -229,9 +228,31 @@ public: ColumnsWithTypeAndName concat_args(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) { - // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; - concat_args[i] = instructions[i].execute(); - // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + const auto & instruction = instructions[i]; + try + { + // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + concat_args[i] = instruction.execute(); + // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + } + catch (const fmt::v9::format_error & e) + { + if (instruction.is_literal) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Bad format {} in function {} without input argument, reason: {}", + instruction.format, + getName(), + e.what()); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Bad format {} in function {} with {} as input argument, reason: {}", + instructions[i].format, + getName(), + instruction.input.dumpStructure(), + e.what()); + } } auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); diff --git a/tests/queries/0_stateless/03203_function_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql index c41cbf0b5e9..6ff4699c8a7 100644 --- a/tests/queries/0_stateless/03203_function_printf.sql +++ b/tests/queries/0_stateless/03203_function_printf.sql @@ -31,4 +31,9 @@ select printf('%%.2e: %.2e', 123.456) = '%.2e: 1.23e+02'; select printf('%%.2g: %.2g', 123.456) = '%.2g: 1.2e+02'; -- Testing character formats with precision -select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; \ No newline at end of file +select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; + +select printf('%%X: %X', 123.123); -- { serverError BAD_ARGUMENTS } +select printf('%%A: %A', 'abc'); -- { serverError BAD_ARGUMENTS } +select printf('%%s: %s', 100); -- { serverError BAD_ARGUMENTS } +select printf('%%n: %n', 100); -- { serverError BAD_ARGUMENTS } From fc651cc0c61feb37e9cf104612cc0ac0cd7448e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 17:57:48 +0200 Subject: [PATCH 444/496] Fix strange code in HostResolvePool --- src/Common/HostResolvePool.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp index cad64ee7204..e8a05a269bc 100644 --- a/src/Common/HostResolvePool.cpp +++ b/src/Common/HostResolvePool.cpp @@ -253,18 +253,18 @@ void HostResolver::updateImpl(Poco::Timestamp now, std::vector Date: Sat, 3 Aug 2024 18:30:33 +0200 Subject: [PATCH 445/496] Fix typo --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 20db4c2773c..7800ee9ff00 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -849,7 +849,7 @@ try #endif #if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + LOG_INFO(log, "Query Profiler is disabled because it cannot work under sanitizers" " when two different stack unwinding methods will interfere with each other."); #endif From 60648e5240fecb92344ff029d2b280f542c3a86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 18:41:17 +0200 Subject: [PATCH 446/496] Revert "Add replication lag and recovery time metrics" --- src/Databases/DatabaseReplicated.cpp | 57 ++++------------- src/Databases/DatabaseReplicated.h | 10 +-- src/Databases/DatabaseReplicatedWorker.cpp | 21 ------- src/Databases/DatabaseReplicatedWorker.h | 5 -- src/Storages/System/StorageSystemClusters.cpp | 37 ++++------- src/Storages/System/StorageSystemClusters.h | 4 +- .../test_recovery_time_metric/__init__.py | 0 .../configs/config.xml | 41 ------------- .../test_recovery_time_metric/test.py | 61 ------------------- .../02117_show_create_table_system.reference | 2 - .../03206_replication_lag_metric.reference | 4 -- .../03206_replication_lag_metric.sql | 11 ---- 12 files changed, 27 insertions(+), 226 deletions(-) delete mode 100644 tests/integration/test_recovery_time_metric/__init__.py delete mode 100644 tests/integration/test_recovery_time_metric/configs/config.xml delete mode 100644 tests/integration/test_recovery_time_metric/test.py delete mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.reference delete mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.sql diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d2dee9b5994..f127ccbc224 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include @@ -339,12 +338,9 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const return std::make_shared(getContext()->getSettingsRef(), shards, params); } -ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const +std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr & cluster_) const { - Strings paths_get, paths_exists; - - paths_get.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); - + Strings paths; const auto & addresses_with_failover = cluster_->getShardsAddresses(); const auto & shards_info = cluster_->getShardsInfo(); for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) @@ -352,59 +348,32 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) for (const auto & replica : addresses_with_failover[shard_index]) { String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths_exists.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); + paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } try { auto current_zookeeper = getZooKeeper(); - auto get_res = current_zookeeper->get(paths_get); - auto exist_res = current_zookeeper->exists(paths_exists); - chassert(get_res.size() == exist_res.size() + 1); + auto res = current_zookeeper->exists(paths); - auto max_log_ptr_zk = get_res[0]; - if (max_log_ptr_zk.error != Coordination::Error::ZOK) - throw Coordination::Exception(max_log_ptr_zk.error); + std::vector statuses; + statuses.resize(paths.size()); - UInt32 max_log_ptr = parse(max_log_ptr_zk.data); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error == Coordination::Error::ZOK) + statuses[i] = 1; - ReplicasInfo replicas_info; - replicas_info.resize(exist_res.size()); - - size_t global_replica_index = 0; - for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) - { - for (const auto & replica : addresses_with_failover[shard_index]) - { - auto replica_active = exist_res[global_replica_index]; - auto replica_log_ptr = get_res[global_replica_index + 1]; - - if (replica_active.error != Coordination::Error::ZOK && replica_active.error != Coordination::Error::ZNONODE) - throw Coordination::Exception(replica_active.error); - - if (replica_log_ptr.error != Coordination::Error::ZOK) - throw Coordination::Exception(replica_log_ptr.error); - - replicas_info[global_replica_index] = ReplicaInfo{ - .is_active = replica_active.error == Coordination::Error::ZOK, - .replication_lag = max_log_ptr - parse(replica_log_ptr.data), - .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, - }; - - ++global_replica_index; - } - } - - return replicas_info; - } catch (...) + return statuses; + } + catch (...) { tryLogCurrentException(log); return {}; } } + void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config_ref) { const auto & config_prefix = fmt::format("named_collections.{}", collection_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 5a1570ae2e2..27ab262d1f1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -17,14 +17,6 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; -struct ReplicaInfo -{ - bool is_active; - UInt32 replication_lag; - UInt64 recovery_time; -}; -using ReplicasInfo = std::vector; - class DatabaseReplicated : public DatabaseAtomic { public: @@ -92,7 +84,7 @@ public: static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, bool throw_if_noop); - ReplicasInfo tryGetReplicasInfo(const ClusterPtr & cluster_) const; + std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; void renameDatabase(ContextPtr query_context, const String & new_name) override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 4e7408aa96e..1ef88dc03bc 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,12 +32,6 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { - { - std::lock_guard lock(initialization_duration_timer_mutex); - initialization_duration_timer.emplace(); - initialization_duration_timer->start(); - } - while (!stop_flag) { try @@ -75,10 +69,6 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - { - std::lock_guard lock(initialization_duration_timer_mutex); - initialization_duration_timer.reset(); - } return true; } catch (...) @@ -88,11 +78,6 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - { - std::lock_guard lock(initialization_duration_timer_mutex); - initialization_duration_timer.reset(); - } - return false; } @@ -474,10 +459,4 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const return max_id.load(); } -UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const -{ - std::lock_guard lock(initialization_duration_timer_mutex); - return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; -} - } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 2309c831839..41edf2221b8 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -36,8 +36,6 @@ public: DatabaseReplicated * const database, bool committed = false); /// NOLINT UInt32 getLogPointer() const; - - UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -58,9 +56,6 @@ private: ZooKeeperPtr active_node_holder_zookeeper; /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; - - std::optional initialization_duration_timer; - mutable std::mutex initialization_duration_timer_mutex; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index d03b600b6ef..160c8d6270e 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -31,8 +31,6 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() {"database_shard_name", std::make_shared(), "The name of the `Replicated` database shard (for clusters that belong to a `Replicated` database)."}, {"database_replica_name", std::make_shared(), "The name of the `Replicated` database replica (for clusters that belong to a `Replicated` database)."}, {"is_active", std::make_shared(std::make_shared()), "The status of the Replicated database replica (for clusters that belong to a Replicated database): 1 means 'replica is online', 0 means 'replica is offline', NULL means 'unknown'."}, - {"replication_lag", std::make_shared(std::make_shared()), "The replication lag of the `Replicated` database replica (for clusters that belong to a Replicated database)."}, - {"recovery_time", std::make_shared(std::make_shared()), "The recovery time of the `Replicated` database replica (for clusters that belong to a Replicated database), in milliseconds."}, }; description.setAliases({ @@ -48,30 +46,31 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co writeCluster(res_columns, name_and_cluster, {}); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & [database_name, database] : databases) + for (const auto & name_and_database : databases) { - if (const auto * replicated = typeid_cast(database.get())) + if (const auto * replicated = typeid_cast(name_and_database.second.get())) { + if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {database_name, database_cluster}, - replicated->tryGetReplicasInfo(database_cluster)); + writeCluster(res_columns, {name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + database_name, database_cluster}, - replicated->tryGetReplicasInfo(database_cluster)); + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); } } } void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const ReplicasInfo & replicas_info) + const std::vector & is_active) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); - size_t global_replica_idx = 0; + size_t replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { const auto & shard_info = shards_info[shard_index]; @@ -100,24 +99,10 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); res_columns[i++]->insert(address.database_shard_name); res_columns[i++]->insert(address.database_replica_name); - if (replicas_info.empty()) - { + if (is_active.empty()) res_columns[i++]->insertDefault(); - res_columns[i++]->insertDefault(); - res_columns[i++]->insertDefault(); - } else - { - const auto & replica_info = replicas_info[global_replica_idx]; - res_columns[i++]->insert(replica_info.is_active); - res_columns[i++]->insert(replica_info.replication_lag); - if (replica_info.recovery_time != 0) - res_columns[i++]->insert(replica_info.recovery_time); - else - res_columns[i++]->insertDefault(); - } - - ++global_replica_idx; + res_columns[i++]->insert(is_active[replica_idx++]); } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index f6e08734896..0f7c792261d 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -1,10 +1,10 @@ #pragma once -#include #include #include #include + namespace DB { @@ -27,7 +27,7 @@ protected: using NameAndCluster = std::pair>; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const ReplicasInfo & replicas_info); + static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); }; } diff --git a/tests/integration/test_recovery_time_metric/__init__.py b/tests/integration/test_recovery_time_metric/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_recovery_time_metric/configs/config.xml b/tests/integration/test_recovery_time_metric/configs/config.xml deleted file mode 100644 index bad9b1fa9ea..00000000000 --- a/tests/integration/test_recovery_time_metric/configs/config.xml +++ /dev/null @@ -1,41 +0,0 @@ - - 9000 - - - - - - - - - default - - - - - - 2181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - 20000 - - - - 1 - localhost - 9444 - - - - - - - localhost - 2181 - - 20000 - - - diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py deleted file mode 100644 index 6fcf2fad423..00000000000 --- a/tests/integration/test_recovery_time_metric/test.py +++ /dev/null @@ -1,61 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node", - main_configs=["configs/config.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_recovery_time_metric(start_cluster): - node.query( - """ - DROP DATABASE IF EXISTS rdb; - CREATE DATABASE rdb - ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') - """ - ) - - node.query( - """ - DROP TABLE IF EXISTS rdb.t; - CREATE TABLE rdb.t - ( - `x` UInt32 - ) - ENGINE = MergeTree - ORDER BY x - """ - ) - - node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) - - node.restart_clickhouse() - - ret = int( - node.query( - """ - SELECT recovery_time - FROM system.clusters - WHERE cluster = 'rdb' - """ - ).strip() - ) - assert ret > 0 - - node.query( - """ - DROP DATABASE rdb - """ - ) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 32e8b2f4312..cfae4fee6c2 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -52,8 +52,6 @@ CREATE TABLE system.clusters `database_shard_name` String, `database_replica_name` String, `is_active` Nullable(UInt8), - `replication_lag` Nullable(UInt32), - `recovery_time` Nullable(UInt64), `name` String ALIAS cluster ) ENGINE = SystemClusters diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.reference b/tests/queries/0_stateless/03206_replication_lag_metric.reference deleted file mode 100644 index 02f4a7264b1..00000000000 --- a/tests/queries/0_stateless/03206_replication_lag_metric.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -2 -0 -2 diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql deleted file mode 100644 index 998c332a11c..00000000000 --- a/tests/queries/0_stateless/03206_replication_lag_metric.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-parallel - -CREATE DATABASE rdb1 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica1'); -CREATE DATABASE rdb2 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica2'); - -SET distributed_ddl_task_timeout = 0; -CREATE TABLE rdb1.t (id UInt32) ENGINE = ReplicatedMergeTree ORDER BY id; -SELECT replication_lag FROM system.clusters WHERE cluster IN ('rdb1', 'rdb2') ORDER BY cluster ASC, replica_num ASC; - -DROP DATABASE rdb1; -DROP DATABASE rdb2; From a19750234153e760907f3c7bc040f949100534df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:11:20 +0200 Subject: [PATCH 447/496] Fix test retries Should fix issues like: - 02494_zero_copy_projection_cancel_fetch - https://s3.amazonaws.com/clickhouse-test-reports/67719/40cd5467c18d65a6624d273ac1a8fd9cc9257d8c/stateless_tests__tsan__s3_storage__[4_4].html Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a29c786e998..877548e577e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2218,7 +2218,6 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) - break except TimeoutError: break finally: From f06ae2f5518ff8cb610b337d4900fd6f0088190f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:27:19 +0200 Subject: [PATCH 448/496] Fill only selected columns from system.clusters Some of them pretty heavy, i.e. is_active for ReplicatedDatabase This should fix 02903_rmt_retriable_merge_exception flakiness [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67687/89c47df559ba23d988f8af3c342e0c8d5531f4b8/fast_test.html Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemClusters.cpp | 82 ++++++++++++------- src/Storages/System/StorageSystemClusters.h | 6 +- 2 files changed, 56 insertions(+), 32 deletions(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 160c8d6270e..9c5c07ae49f 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -40,10 +40,10 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() return description; } -void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const { for (const auto & name_and_cluster : context->getClusters()) - writeCluster(res_columns, name_and_cluster, {}); + writeCluster(res_columns, columns_mask, name_and_cluster, /* replicated= */ nullptr); const auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & name_and_database : databases) @@ -52,18 +52,15 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co { if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {name_and_database.first, database_cluster}, replicated); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, replicated); } } } -void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const std::vector & is_active) +void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; @@ -79,30 +76,55 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index) { - size_t i = 0; + size_t src_index = 0, res_index = 0; const auto & address = shard_addresses[replica_index]; - res_columns[i++]->insert(cluster_name); - res_columns[i++]->insert(shard_info.shard_num); - res_columns[i++]->insert(shard_info.weight); - res_columns[i++]->insert(shard_info.has_internal_replication); - res_columns[i++]->insert(replica_index + 1); - res_columns[i++]->insert(address.host_name); - auto resolved = address.getResolvedAddress(); - res_columns[i++]->insert(resolved ? resolved->host().toString() : String()); - res_columns[i++]->insert(address.port); - res_columns[i++]->insert(address.is_local); - res_columns[i++]->insert(address.user); - res_columns[i++]->insert(address.default_database); - res_columns[i++]->insert(pool_status[replica_index].error_count); - res_columns[i++]->insert(pool_status[replica_index].slowdown_count); - res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); - res_columns[i++]->insert(address.database_shard_name); - res_columns[i++]->insert(address.database_replica_name); - if (is_active.empty()) - res_columns[i++]->insertDefault(); - else - res_columns[i++]->insert(is_active[replica_idx++]); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(cluster_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.shard_num); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.weight); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.has_internal_replication); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(replica_index + 1); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.host_name); + if (columns_mask[src_index++]) + { + auto resolved = address.getResolvedAddress(); + res_columns[res_index++]->insert(resolved ? resolved->host().toString() : String()); + } + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.port); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.is_local); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.user); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.default_database); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].error_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].slowdown_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].estimated_recovery_time.count()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.database_shard_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.database_replica_name); + if (columns_mask[src_index++]) + { + std::vector is_active; + if (replicated) + is_active = replicated->tryGetAreReplicasActive(name_and_cluster.second); + + if (is_active.empty()) + res_columns[res_index++]->insertDefault(); + else + res_columns[res_index++]->insert(is_active[replica_idx++]); + } } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 0f7c792261d..f6adb902f43 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -10,6 +10,7 @@ namespace DB class Context; class Cluster; +class DatabaseReplicated; /** Implements system table 'clusters' * that allows to obtain information about available clusters @@ -26,8 +27,9 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; using NameAndCluster = std::pair>; - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const override; + static void writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated); + bool supportsColumnsMask() const override { return true; } }; } From 9d0e066cda8d0ccb6bd4f9e07fee36a2bfae707a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:41:10 +0200 Subject: [PATCH 449/496] Bump NuRaft (to properly catch thread exceptions) Refs: https://github.com/ClickHouse/NuRaft/pull/75 Refs: https://github.com/eBay/NuRaft/pull/525 Signed-off-by: Azat Khuzhin --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce From 8562a6106c286882f26383086e52e399106893be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:45:18 +0200 Subject: [PATCH 450/496] Better safety thresholds in `arrayWithConstant` --- src/Functions/array/arrayWithConstant.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayWithConstant.cpp b/src/Functions/array/arrayWithConstant.cpp index 48262870553..4cbc6404b9b 100644 --- a/src/Functions/array/arrayWithConstant.cpp +++ b/src/Functions/array/arrayWithConstant.cpp @@ -1,9 +1,9 @@ #include -#include #include #include #include #include +#include namespace DB @@ -15,7 +15,8 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } -/// Reasonable threshold. +/// Reasonable thresholds. +static constexpr Int64 max_array_size_in_columns_bytes = 1000000000; static constexpr size_t max_arrays_size_in_columns = 1000000000; @@ -63,12 +64,19 @@ public: auto array_size = col_num->getInt(i); if (unlikely(array_size < 0)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size cannot be negative: while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} cannot be negative: while executing function {}", array_size, getName()); + + Int64 estimated_size = 0; + if (unlikely(common::mulOverflow(array_size, col_value->byteSize(), estimated_size))) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); + + if (unlikely(estimated_size > max_array_size_in_columns_bytes)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); offset += array_size; if (unlikely(offset > max_arrays_size_in_columns)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size {} (will generate at least {} elements) while executing function {}", array_size, offset, getName()); offsets.push_back(offset); } From 7a066a6505108b14bd49da8766c1bc473a978b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:48:51 +0200 Subject: [PATCH 451/496] Add a test --- .../0_stateless/03216_arrayWithConstant_limits.reference | 1 + tests/queries/0_stateless/03216_arrayWithConstant_limits.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.reference create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.sql diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference new file mode 100644 index 00000000000..825319e1c5b --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference @@ -0,0 +1 @@ +10000000 diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql new file mode 100644 index 00000000000..c46524c50e6 --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql @@ -0,0 +1,3 @@ +SELECT arrayWithConstant(96142475, ['qMUF']); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT arrayWithConstant(100000000, materialize([[[[[[[[[['Hello, world!']]]]]]]]]])); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT length(arrayWithConstant(10000000, materialize([[[[[[[[[['Hello world']]]]]]]]]]))); From 6013e4b81d29f2b46bec64859132a4e2ff1bbdc9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Jul 2024 20:52:48 +0200 Subject: [PATCH 452/496] poco/MongoDB: Support broader UUID types (generated by python uuid.UUID) Signed-off-by: Azat Khuzhin --- base/poco/MongoDB/src/Binary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/MongoDB/src/Binary.cpp b/base/poco/MongoDB/src/Binary.cpp index ea814d6969f..47f3453630f 100644 --- a/base/poco/MongoDB/src/Binary.cpp +++ b/base/poco/MongoDB/src/Binary.cpp @@ -76,7 +76,7 @@ std::string Binary::toString(int indent) const UUID Binary::uuid() const { - if (_subtype == 0x04 && _buffer.size() == 16) + if ((_subtype == 0x04 || _subtype == 0x03) && _buffer.size() == 16) { UUID uuid; uuid.copyFrom((const char*) _buffer.begin()); From 2f00c962711e13ca00af324366421fe4593b4ce6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Jul 2024 20:34:15 +0200 Subject: [PATCH 453/496] Support true UUID type for MongoDB engine Signed-off-by: Azat Khuzhin --- base/poco/Foundation/include/Poco/UUID.h | 7 +++ base/poco/MongoDB/src/Binary.cpp | 2 +- src/Processors/Sources/MongoDBSource.cpp | 46 +++++++++++++++++-- .../integration/test_storage_mongodb/test.py | 33 +++++++++++++ 4 files changed, 82 insertions(+), 6 deletions(-) diff --git a/base/poco/Foundation/include/Poco/UUID.h b/base/poco/Foundation/include/Poco/UUID.h index df67ef73e4b..6466d226b2e 100644 --- a/base/poco/Foundation/include/Poco/UUID.h +++ b/base/poco/Foundation/include/Poco/UUID.h @@ -19,6 +19,7 @@ #include "Poco/Foundation.h" +#include namespace Poco @@ -135,6 +136,12 @@ public: static const UUID & x500(); /// Returns the namespace identifier for the X500 namespace. + UInt32 getTimeLow() const { return _timeLow; } + UInt16 getTimeMid() const { return _timeMid; } + UInt16 getTimeHiAndVersion() const { return _timeHiAndVersion; } + UInt16 getClockSeq() const { return _clockSeq; } + std::array getNode() const { return std::array{_node[0], _node[1], _node[2], _node[3], _node[4], _node[5]}; } + protected: UUID(UInt32 timeLow, UInt32 timeMid, UInt32 timeHiAndVersion, UInt16 clockSeq, UInt8 node[]); UUID(const char * bytes, Version version); diff --git a/base/poco/MongoDB/src/Binary.cpp b/base/poco/MongoDB/src/Binary.cpp index 47f3453630f..8b0e6baeccb 100644 --- a/base/poco/MongoDB/src/Binary.cpp +++ b/base/poco/MongoDB/src/Binary.cpp @@ -82,7 +82,7 @@ UUID Binary::uuid() const uuid.copyFrom((const char*) _buffer.begin()); return uuid; } - throw BadCastException("Invalid subtype"); + throw BadCastException("Invalid subtype: " + std::to_string(_subtype) + ", size: " + std::to_string(_buffer.size())); } diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index 0d583cf6be5..e00a541b300 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -17,6 +18,7 @@ #include #include #include +#include "base/types.h" #include #include @@ -45,8 +47,28 @@ namespace using ValueType = ExternalResultDescription::ValueType; using ObjectId = Poco::MongoDB::ObjectId; using MongoArray = Poco::MongoDB::Array; + using MongoUUID = Poco::MongoDB::Binary::Ptr; + UUID parsePocoUUID(const Poco::UUID & src) + { + UUID uuid; + + std::array src_node = src.getNode(); + UInt64 node = 0; + node |= UInt64(src_node[0]) << 40; + node |= UInt64(src_node[1]) << 32; + node |= UInt64(src_node[2]) << 24; + node |= UInt64(src_node[3]) << 16; + node |= UInt64(src_node[4]) << 8; + node |= src_node[5]; + + UUIDHelpers::getHighBytes(uuid) = UInt64(src.getTimeLow()) << 32 | UInt32(src.getTimeMid() << 16 | src.getTimeHiAndVersion()); + UUIDHelpers::getLowBytes(uuid) = UInt64(src.getClockSeq()) << 48 | node; + + return uuid; + } + template Field getNumber(const Poco::MongoDB::Element & value, const std::string & name) { @@ -149,12 +171,20 @@ namespace else if (which.isUUID()) parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String (UUID), got type id = {} for column {}", + if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + String string = static_cast &>(value).value(); + return parse(string); + } + else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); + return parsePocoUUID(poco_uuid); + } + else + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", toString(value.type()), name); - String string = static_cast &>(value).value(); - return parse(string); }; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); @@ -286,8 +316,14 @@ namespace String string = static_cast &>(value).value(); assert_cast(column).getData().push_back(parse(string)); } + else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); + UUID uuid = parsePocoUUID(poco_uuid); + assert_cast(column).getData().push_back(uuid); + } else - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String (UUID), got type id = {} for column {}", + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", toString(value.type()), name); break; } diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 3957afe8b29..1a1a790e8e8 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -1,4 +1,5 @@ import pymongo +from uuid import UUID import pytest from helpers.client import QueryRuntimeException @@ -72,6 +73,28 @@ def test_simple_select(started_cluster): simple_mongo_table.drop() +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_uuid(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + mongo_table = db["uuid_table"] + mongo_table.insert({"key": 0, "data": UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed")}) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE uuid_mongo_table(key UInt64, data UUID) ENGINE = MongoDB('mongo1:27017', 'test', 'uuid_table', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM uuid_mongo_table") == "1\n" + assert ( + node.query("SELECT data from uuid_mongo_table where key = 0") + == "f0e77736-91d1-48ce-8f01-15123ca1c7ed\n" + ) + node.query("DROP TABLE uuid_mongo_table") + mongo_table.drop() + + @pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) def test_simple_select_from_view(started_cluster): mongo_connection = get_mongo_connection(started_cluster) @@ -140,6 +163,10 @@ def test_arrays(started_cluster): "f0e77736-91d1-48ce-8f01-15123ca1c7ed", "93376a07-c044-4281-a76e-ad27cf6973c5", ], + "arr_mongo_uuid": [ + UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed"), + UUID("93376a07-c044-4281-a76e-ad27cf6973c5"), + ], "arr_arr_bool": [ [True, False, True], [True], @@ -174,6 +201,7 @@ def test_arrays(started_cluster): "arr_datetime Array(DateTime)," "arr_string Array(String)," "arr_uuid Array(UUID)," + "arr_mongo_uuid Array(UUID)," "arr_arr_bool Array(Array(Bool))," "arr_empty Array(UInt64)," "arr_null Array(UInt64)," @@ -222,6 +250,11 @@ def test_arrays(started_cluster): == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" ) + assert ( + node.query(f"SELECT arr_mongo_uuid FROM arrays_mongo_table WHERE key = 42") + == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" + ) + assert ( node.query(f"SELECT arr_arr_bool FROM arrays_mongo_table WHERE key = 42") == "[[true,false,true],[true],[],[],[false],[false]]\n" From a051eb2a5d1c29a61d3c2d9e2f3f0841bbb2817c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 00:24:17 +0200 Subject: [PATCH 454/496] Fix tests --- src/Databases/DatabaseHDFS.cpp | 2 +- src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index f58f1b76e71..7fa67a5678e 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -51,7 +51,7 @@ DatabaseHDFS::DatabaseHDFS(const String & name_, const String & source_url, Cont if (!source.empty()) { if (!re2::RE2::FullMatch(source, std::string(HDFS_HOST_REGEXP))) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs host: {}. " + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS host: {}. " "It should have structure 'hdfs://:'", source); context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(source)); diff --git a/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp index 365828bcc40..7f8727eea1c 100644 --- a/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp +++ b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp @@ -192,7 +192,7 @@ String getNameNodeCluster(const String &hdfs_url) void checkHDFSURL(const String & url) { if (!re2::RE2::FullMatch(url, std::string(HDFS_URL_REGEXP))) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. It should have structure 'hdfs://:/'", url); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have structure 'hdfs://:/'", url); } } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 47d8f44c0b7..ccd2c7eaf11 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -157,7 +157,7 @@ def test_bad_hdfs_uri(started_cluster): ) except Exception as ex: print(ex) - assert "Bad hdfs url" in str(ex) + assert "Bad HDFS URL" in str(ex) try: node1.query( "create table BadStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs100500:9000/other_storage', 'TSV')" From 185b6a54da8f24a97f130bdebe7bb1ec2bd266c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 00:34:58 +0200 Subject: [PATCH 455/496] Merge with master --- src/Core/SettingsChangesHistory.cpp | 261 +--------------------------- 1 file changed, 1 insertion(+), 260 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3f07bfdb933..5b94391bade 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.8", {{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."}}}, - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_reader_time_zone_name", "GMT", "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -335,6 +75,7 @@ static std::initializer_list Date: Sun, 4 Aug 2024 09:02:19 +0200 Subject: [PATCH 456/496] Use RabbitMQ without management in tests (attempt to improve startup) Sometimes startup can take ~90 seconds [1]: 2024-08-03 23:11:38.756067+00:00 [info] <0.9.0> Time to start RabbitMQ: 94651980 us Unlike normally ~10 [2]: 2024-08-04 00:33:07.016137+00:00 [info] <0.9.0> Time to start RabbitMQ: 10082489 us [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/b4e3bbcb82158bea4f5db1d9f5c28cfb741d1d51/integration_tests__asan__old_analyzer__[4_6].html [2]: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__asan__old_analyzer__[4_6].html I've tried locally, the difference is very small, 3135665 us (+management) vs (2740747 us), but still something, and who knows how it works under pressure. Signed-off-by: Azat Khuzhin --- tests/integration/compose/docker_compose_rabbitmq.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_rabbitmq.yml b/tests/integration/compose/docker_compose_rabbitmq.yml index 61b21e0e3d9..94c7f0111c4 100644 --- a/tests/integration/compose/docker_compose_rabbitmq.yml +++ b/tests/integration/compose/docker_compose_rabbitmq.yml @@ -2,7 +2,7 @@ version: '2.3' services: rabbitmq1: - image: rabbitmq:3.12.6-management-alpine + image: rabbitmq:3.12.6-alpine hostname: rabbitmq1 expose: - ${RABBITMQ_PORT:-5672} From 264be9c598b42d91ee0a19f718c4d9a4291c7bc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:06:50 +0200 Subject: [PATCH 457/496] tests: increase timeout for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9259c720ff0..a1bdee33d57 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2371,7 +2371,7 @@ class ClickHouseCluster: time.sleep(0.5) raise Exception("Cannot wait PostgreSQL Java Client container") - def wait_rabbitmq_to_start(self, timeout=30): + def wait_rabbitmq_to_start(self, timeout=60): self.print_all_docker_pieces() self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) From dc527b6fd1dfdffb1d177237bffc69cd110cd2a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:07:05 +0200 Subject: [PATCH 458/496] tests: detailed errors for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a1bdee33d57..56d111629c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2399,7 +2399,7 @@ class ClickHouseCluster: ) rabbitmq_debuginfo(self.rabbitmq_docker_id, self.rabbitmq_cookie) except Exception as e: - logging.debug("Unable to get logs from docker.") + logging.debug(f"Unable to get logs from docker: {e}.") raise Exception("Cannot wait RabbitMQ container") def wait_nats_is_available(self, max_retries=5): From 27db715761ec66b0dfc915269270697e6cf67909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 4 Aug 2024 09:52:36 +0000 Subject: [PATCH 459/496] Incorporate review feedback --- src/Functions/CountSubstringsImpl.h | 16 +++++++++----- src/Functions/MatchImpl.h | 34 ++++++++++++++--------------- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 8e91bc3aeb4..b1cefae6f1d 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -38,10 +38,10 @@ struct CountSubstringsImpl const ColumnPtr & start_pos, PaddedPODArray & res, [[maybe_unused]] ColumnUInt8 * res_null, - size_t /*input_rows_count*/) + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); const UInt8 * const begin = haystack_data.data(); const UInt8 * const end = haystack_data.data() + haystack_data.size(); @@ -81,6 +81,8 @@ struct CountSubstringsImpl } pos = begin + haystack_offsets[i]; ++i; + + chassert(i < input_rows_count); } } @@ -116,7 +118,7 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); Impl::toLowerIfNeed(haystack); Impl::toLowerIfNeed(needle); @@ -154,8 +156,10 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null, size_t input_rows_count) { + chassert(input_rows_count == haystack_offsets.size()); + /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; @@ -210,8 +214,10 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null, size_t input_rows_count) { + chassert(input_rows_count == needle_offsets.size()); + /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index ceac753fe79..7dc93ba79e0 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -131,11 +131,11 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == haystack_offsets.size()); - assert(res.size() == input_rows_count); - assert(start_pos_ == nullptr); + chassert(res.size() == haystack_offsets.size()); + chassert(res.size() == input_rows_count); + chassert(start_pos_ == nullptr); if (input_rows_count == 0) return; @@ -295,10 +295,10 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == haystack.size() / N); - assert(res.size() == input_rows_count); + chassert(res.size() == haystack.size() / N); + chassert(res.size() == input_rows_count); if (input_rows_count == 0) return; @@ -468,12 +468,12 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(haystack_offsets.size() == needle_offset.size()); - assert(res.size() == haystack_offsets.size()); - assert(res.size() == input_rows_count); - assert(start_pos_ == nullptr); + chassert(haystack_offsets.size() == needle_offset.size()); + chassert(res.size() == haystack_offsets.size()); + chassert(res.size() == input_rows_count); + chassert(start_pos_ == nullptr); if (input_rows_count == 0) return; @@ -577,12 +577,12 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == input_rows_count); - assert(res.size() == haystack.size() / N); - assert(res.size() == needle_offset.size()); - assert(start_pos_ == nullptr); + chassert(res.size() == input_rows_count); + chassert(res.size() == haystack.size() / N); + chassert(res.size() == needle_offset.size()); + chassert(start_pos_ == nullptr); if (haystack.empty()) return; From f0aaac3bd19c21a796f54ba080fd67f92959131a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:14:28 +0200 Subject: [PATCH 460/496] tests: remove useless retries from test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 3b79ea7916d..48a6224347d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -76,7 +76,7 @@ def get_used_disks_for_table(node, table_name, partition=None): ) -def check_used_disks_with_retry(node, table_name, expected_disks, retries): +def check_used_disks_with_retry(node, table_name, expected_disks, retries=1): for _ in range(retries): used_disks = get_used_disks_for_table(node, table_name) if set(used_disks).issubset(expected_disks): @@ -1635,9 +1635,9 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"]), 100) + assert check_used_disks_with_retry(node1, name, set(["external"])) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"]), 50) + assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) time.sleep(5) From 47dbc5e05b12213a08c25ade9536603a3fd2b175 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:16:17 +0200 Subject: [PATCH 461/496] tests: add debug info into test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 48a6224347d..4ebe9a30699 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1635,9 +1635,17 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"])) + assert check_used_disks_with_retry( + node1, name, set(["external"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) + assert check_used_disks_with_retry( + node1, name, set(["jbod1", "jbod2"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) time.sleep(5) From 062490e1b40a8df8d63fca567b11e7dd26cf52ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:22:24 +0200 Subject: [PATCH 462/496] tests: fix test_ttl_move::test_alter_with_merge_work flakiness MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Increase timeout for TTL DELETE, since otherwise if other routines will take too long, the part will be removed when it should be still be on "external" disk: 2024.08.04 03:48:53.803032 [ 622 ] {} default.mt_test_alter_with_merge_work_1722743323 (9dc6904a-f082-4f06-be7a-efe4733e811c): Will drop empty part all_1_3_4_4 And this is how part_log looks like: SELECT event_time, event_type, rows, part_name, error, database, disk_name FROM system.part_log WHERE `table` = 'mt_test_alter_with_merge_work_1722743323' ORDER BY event_time ASC Query id: a118b3cd-e4fe-45a5-b675-d73bdd887d79 ┌──────────event_time─┬─event_type─┬─rows─┬─part_name───┬─error─┬─database─┬─disk_name─┐ 1. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_1_1_0 │ 0 │ default │ jbod1 │ 2. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_2_2_0 │ 0 │ default │ jbod2 │ 3. │ 2024-08-04 03:48:45 │ NewPart │ 2 │ all_3_3_0 │ 0 │ default │ jbod1 │ 4. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_1_1_0_4 │ 0 │ default │ jbod1 │ 5. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_2_2_0_4 │ 0 │ default │ jbod2 │ 6. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod1 │ 7. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_1_1_0_4 │ 0 │ default │ external │ 8. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod2 │ 9. │ 2024-08-04 03:48:47 │ MergeParts │ 6 │ all_1_3_1_4 │ 0 │ default │ jbod2 │ 10. │ 2024-08-04 03:48:48 │ MovePart │ 6 │ all_1_3_1_4 │ 0 │ default │ external │ 11. │ 2024-08-04 03:48:52 │ MergeParts │ 4 │ all_1_3_2_4 │ 0 │ default │ external │ 12. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_3_4 │ 0 │ default │ external │ # rows==0 13. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_4_4 │ 0 │ default │ external │ └─────────────────────┴────────────┴──────┴─────────────┴───────┴──────────┴───────────┘ CI: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__tsan__[5_6].html Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 4ebe9a30699..925bdf9baaa 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1613,7 +1613,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): ALTER TABLE {name} MODIFY TTL d1 + INTERVAL 0 SECOND TO DISK 'jbod2', d1 + INTERVAL 5 SECOND TO VOLUME 'external', - d1 + INTERVAL 10 SECOND DELETE + d1 + INTERVAL 30 SECOND DELETE """.format( name=name ) @@ -1647,7 +1647,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" ) - time.sleep(5) + time.sleep(25) optimize_table(20) From 9f31488e502c2b2c02e3058f9794829aac14f8b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 15:43:18 +0200 Subject: [PATCH 463/496] Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading On CI you can find that 01747_executable_pool_dictionary_implicit_key can hang [1], it is possible due to after CANNOT_SCHEDULE_TASK the async loading will hang: 2024.07.18 03:56:32.365226 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} ExternalDictionariesLoader: Will load the object 'executable_pool_simple_implicit_key' in background, force = false, loading_id = 2 2024.07.18 03:56:32.368005 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} executeQuery: Code: 439. DB::Exception: Cannot schedule a task: fault injected (threads=766, jobs=746): In scope SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)). (CANNOT_SCHEDULE_TASK) (version 24.7.1.2241) (from [::1]:56446) (comment: 01747_executable_pool_dictionary_implicit_key.sql) (in query: SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1));), Stack trace (when copying this message, always include the lines below): 0. /build/contrib/llvm-project/libcxx/include/exception:141: Poco::Exception::Exception(String const&, int) @ 0x0000000015f8a292 1. /build/src/Common/Exception.cpp:110: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c3df6b9 2. /build/contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x0000000006de714c 3. /build/contrib/llvm-project/libcxx/include/vector:438: DB::Exception::Exception(int, FormatStringHelperImpl::type, std::type_identity::type, std::type_identity::type>, String const&, unsigned long&&, unsigned long&) @ 0x000000000c4838eb 4. /build/src/Common/ThreadPool.cpp:0: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda'(String const&)::operator()(String const&) const @ 0x000000000c4832d3 5. /build/src/Common/ThreadPool.cpp:186: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool) @ 0x000000000c47e7db 6. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000000c47ec8d 7. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001114b16e 8. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::ExternalLoader::LoadingDispatcher::startLoading(DB::ExternalLoader::LoadingDispatcher::Info&, bool, unsigned long) @ 0x0000000011147733 9. /build/src/Interpreters/ExternalLoader.cpp:837: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&)::'lambda'()::operator()() const @ 0x0000000011158bf9 10. /build/contrib/llvm-project/libcxx/include/__mutex_base:397: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&) @ 0x00000000111588bc 11. /build/src/Interpreters/ExternalLoader.cpp:604: DB::ExternalLoader::LoadResult DB::ExternalLoader::LoadingDispatcher::tryLoad(String const&, std::chrono::duration>) @ 0x00000000111440bf 12. /build/src/Interpreters/ExternalLoader.cpp:1381: std::shared_ptr DB::ExternalLoader::load, void>(String const&) const @ 0x00000000111442f5 13. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:587: DB::ExternalDictionariesLoader::getDictionary(String const&, std::shared_ptr) const @ 0x0000000011141028 14. /build/src/Functions/FunctionsExternalDictionaries.h:76: DB::FunctionDictHelper::getDictionary(String const&) @ 0x00000000071d28ec ... 2024.07.18 03:58:29.000900 [ 48468 ] {8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b} executeQuery: (from [::1]:40410) (comment: 01747_executable_pool_dictionary_implicit_key.sql) SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)); (stage: Complete) # and no more rows for 8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b [1]: https://s3.amazonaws.com/clickhouse-test-reports/66495/bc029ed8207ac75e96e9cb48cb79d27a9ffa4e2f/stress_test__debug_.html The problem that it should be properly cancelled, otherwise it will not be loaded in loadImpl(), but will be waited. Signed-off-by: Azat Khuzhin --- src/Interpreters/ExternalLoader.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 96405f35f3f..511300be2e0 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -922,7 +922,16 @@ private: if (enable_async_loading) { /// Put a job to the thread pool for the loading. - auto thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + ThreadFromGlobalPool thread; + try + { + thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + } + catch (...) + { + cancelLoading(info); + throw; + } loading_threads.try_emplace(loading_id, std::move(thread)); } else From dea95e6c332392a87ec3175bb32a50770a57d65b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:06:07 +0200 Subject: [PATCH 464/496] Ensure that llvm-symbolizer is used for symbolizing sanitizer reports Since you don't want to fallback to addr2line: # addr2line $ time /bin/test set_flag_impl: Success set_flag_if: Success /usr/bin/addr2line: DWARF error: invalid or unhandled FORM value: 0x23 ================== WARNING: ThreadSanitizer: data race (pid=18) ... real 3m8.580s user 0m21.967s sys 0m40.628s # llvm-symbolizer $ time ./test set_flag_impl: Success set_flag_if: Success ================== WARNING: ThreadSanitizer: data race (pid=24884) real 0m0.028s user 0m0.003s sys 0m0.006s Signed-off-by: Azat Khuzhin --- docker/test/base/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index a81826ed6b5..e7ac62604dc 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -35,7 +35,9 @@ ENV UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768' ENV LSAN_OPTIONS='max_allocation_size_mb=32768' -# for external_symbolizer_path +# for external_symbolizer_path, and also ensure that llvm-symbolizer really +# exists (since you don't want to fallback to addr2line, it is very slow) +RUN test -f /usr/bin/llvm-symbolizer-${LLVM_VERSION} RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 From 1f2bb6676f39259ec20dd0b077381d251b591d3a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:08:09 +0200 Subject: [PATCH 465/496] Remove outdated comments about workaround for #64086 Signed-off-by: Azat Khuzhin --- docker/test/util/Dockerfile | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index babddbd228c..dc928ba7195 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -5,14 +5,6 @@ FROM ubuntu:22.04 ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -# FIXME: rebuild for clang 18.1.3, that contains a workaround [1] for -# sanitizers issue [2]: -# -# $ git tag --contains c2a57034eff048cd36c563c8e0051db3a70991b3 | tail -1 -# llvmorg-18.1.3 -# -# [1]: https://github.com/llvm/llvm-project/commit/c2a57034eff048cd36c563c8e0051db3a70991b3 -# [2]: https://github.com/ClickHouse/ClickHouse/issues/64086 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ From eed2edd7db5f2424d85a66bfee63281cba5a5c94 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 Jul 2024 12:23:10 +0200 Subject: [PATCH 466/496] Comment certificateFile/privateKeyFile/dhParamsFile in keeper config Otherwise you will get annoying messages at startup: 2024.07.02 10:03:38.331593 [ 1 ] {} CertificateReloader: Cannot obtain modification time for certificate file /etc/clickhouse-keeper/server.crt, skipping update. errno: 2, strerror: 0 2024.07.02 10:03:38.331658 [ 1 ] {} CertificateReloader: Cannot obtain modification time for key file /etc/clickhouse-keeper/server.key, skipping update. errno: 2, strerror: 0 2024.07.02 10:03:38.341085 [ 1 ] {} CertificateReloader: Poco::Exception. Code: 1000, e.code() = 0, SSL context exception: Error loading private key from file /etc/clickhouse-keeper/server.key: error:80000002:system library::No such file or directory Signed-off-by: Azat Khuzhin --- programs/keeper/keeper_config.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/keeper/keeper_config.xml b/programs/keeper/keeper_config.xml index 4cf84cffc86..efd0010d184 100644 --- a/programs/keeper/keeper_config.xml +++ b/programs/keeper/keeper_config.xml @@ -66,14 +66,14 @@ - /etc/clickhouse-keeper/server.crt - /etc/clickhouse-keeper/server.key + + - /etc/clickhouse-keeper/dhparam.pem + none true true From a75c4b9f9fe519cb74a47d88efb55216da22a64b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:07:40 +0200 Subject: [PATCH 467/496] Introduce `no-flaky-check` tag --- tests/clickhouse-test | 3 +++ .../00002_log_and_exception_messages_formatting.sql | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 893fded8d23..b70dd61a25a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,6 +1232,9 @@ class TestCase: ): return FailureReason.SKIP + elif "no-flaky-check" in tags and (args.test_runs > 1): + return FailureReason.SKIP + elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 07c42d6d039..32db77cd8dd 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-ubsan, no-batch +-- Tags: no-parallel, no-fasttest, no-ubsan, no-batch, no-flaky-check -- no-parallel because we want to run this test when most of the other tests already passed -- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log From 87e0cf6b7635b7249d72c6ac0e7aad75c863d01d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:25:36 +0200 Subject: [PATCH 468/496] Fix stack overflow in JSONMergePatch --- src/Common/JSONParsers/RapidJSONParser.h | 12 ++++++--- src/Functions/jsonMergePatch.cpp | 34 +++++++++++++++--------- 2 files changed, 29 insertions(+), 17 deletions(-) diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 6c5ea938bfe..ad7a4cbf53a 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -3,10 +3,14 @@ #include "config.h" #if USE_RAPIDJSON -# include -# include -# include -# include "ElementTypes.h" + +/// Prevent stack overflow: +#define RAPIDJSON_PARSE_DEFAULT_FLAGS (kParseIterativeFlag) + +#include +#include +#include +#include "ElementTypes.h" namespace DB { diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index a83daacdbf6..3bde415aabf 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -10,12 +10,14 @@ #if USE_RAPIDJSON -#include "rapidjson/document.h" -#include "rapidjson/writer.h" -#include "rapidjson/stringbuffer.h" -#include "rapidjson/filewritestream.h" -#include "rapidjson/prettywriter.h" -#include "rapidjson/filereadstream.h" +/// Prevent stack overflow: +#define RAPIDJSON_PARSE_DEFAULT_FLAGS (kParseIterativeFlag) + +#include +#include +#include +#include +#include namespace DB @@ -31,17 +33,17 @@ namespace ErrorCodes namespace { - // select jsonMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); + // select JSONMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); // || // \/ // ┌───────────────────────┐ // │ {"a":1,"name":"zoey"} │ // └───────────────────────┘ - class FunctionjsonMergePatch : public IFunction + class FunctionJSONMergePatch : public IFunction { public: - static constexpr auto name = "jsonMergePatch"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static constexpr auto name = "JSONMergePatch"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -98,7 +100,11 @@ namespace const char * json = str_ref.data; document.Parse(json); - if (document.HasParseError() || !document.IsObject()) + + if (document.HasParseError()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong JSON string to merge: {}", rapidjson::GetParseError_En(document.GetParseError())); + + if (!document.IsObject()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong JSON string to merge. Expected JSON object"); }; @@ -162,10 +168,12 @@ namespace } -REGISTER_FUNCTION(jsonMergePatch) +REGISTER_FUNCTION(JSONMergePatch) { - factory.registerFunction(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description="Returns the merged JSON object string, which is formed by merging multiple JSON objects."}); + + factory.registerAlias("jsonMergePatch", "JSONMergePatch"); } } From 37da0b3c307d249a078b94cb7bc53d8acdd22a98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:27:37 +0200 Subject: [PATCH 469/496] Add a test --- .../03217_json_merge_patch_stack_overflow.reference | 0 .../03217_json_merge_patch_stack_overflow.sql | 9 +++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference create mode 100644 tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql diff --git a/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql new file mode 100644 index 00000000000..4b366b08c6b --- /dev/null +++ b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql @@ -0,0 +1,9 @@ +-- Tags: no-fasttest +-- Needs rapidjson library +SELECT JSONMergePatch(REPEAT('{"c":', 1000000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 100000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 10000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 1000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 100)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 10)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 1)); -- { serverError BAD_ARGUMENTS } From a433115434ebe4c1d69f2ed200005fa93c7adcb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 19:18:00 +0200 Subject: [PATCH 470/496] Fix typo --- docker/test/util/process_functional_tests_result.py | 6 +++--- docs/changelogs/v20.5.1.3833-prestable.md | 2 +- docs/changelogs/v21.11.1.8636-prestable.md | 6 +++--- docs/changelogs/v21.12.1.9017-prestable.md | 10 +++++----- docs/changelogs/v21.5.1.6601-prestable.md | 2 +- docs/changelogs/v21.6.9.7-stable.md | 2 +- docs/changelogs/v21.7.9.7-stable.md | 2 +- docs/changelogs/v21.8.1.7409-prestable.md | 2 +- docs/changelogs/v21.8.5.7-lts.md | 2 +- docs/changelogs/v21.9.1.8000-prestable.md | 2 +- docs/changelogs/v22.1.1.2542-prestable.md | 4 ++-- docs/changelogs/v22.4.1.2305-prestable.md | 2 +- docs/changelogs/v23.4.1.1943-stable.md | 3 +-- docs/changelogs/v23.6.1.1524-stable.md | 2 +- docs/zh/changelog/index.md | 2 +- src/Common/SystemLogBase.cpp | 2 +- 16 files changed, 25 insertions(+), 26 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 3da1a8f3674..aa2ea686c46 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -161,11 +161,11 @@ def process_result(result_path, broken_tests): retries, test_results, ) = process_test_log(result_path, broken_tests) - is_flacky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) - logging.info("Is flaky check: %s", is_flacky_check) + is_flaky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) + logging.info("Is flaky check: %s", is_flaky_check) # If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately) # But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped. - if failed != 0 or unknown != 0 or (success == 0 and (not is_flacky_check)): + if failed != 0 or unknown != 0 or (success == 0 and (not is_flaky_check)): state = "failure" if hung: diff --git a/docs/changelogs/v20.5.1.3833-prestable.md b/docs/changelogs/v20.5.1.3833-prestable.md index 79d61fb12f9..3de263f5e37 100644 --- a/docs/changelogs/v20.5.1.3833-prestable.md +++ b/docs/changelogs/v20.5.1.3833-prestable.md @@ -331,7 +331,7 @@ * Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). * Add a test for Join table engine from @donmikel. This closes [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Repeat test in CI if `curl` invocation was timed out. It is possible due to system hangups for 10+ seconds that are typical in our CI infrastructure. This fixes [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix potentially flacky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix potentially flaky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Now clickhouse-test check the server aliveness before tests run. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). * Emit a warning if server was build in debug or with sanitizers. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Better check for hung queries in clickhouse-test. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v21.11.1.8636-prestable.md b/docs/changelogs/v21.11.1.8636-prestable.md index d6a435dd3ce..95c8580d591 100644 --- a/docs/changelogs/v21.11.1.8636-prestable.md +++ b/docs/changelogs/v21.11.1.8636-prestable.md @@ -280,7 +280,7 @@ sidebar_label: 2022 * Cleanup unbundled image [#29689](https://github.com/ClickHouse/ClickHouse/pull/29689) ([Azat Khuzhin](https://github.com/azat)). * Fix memory tracking for merges and mutations [#29691](https://github.com/ClickHouse/ClickHouse/pull/29691) ([Azat Khuzhin](https://github.com/azat)). * Fix data-race in WriteIndirectBuffer (used in DiskMemory) [#29692](https://github.com/ClickHouse/ClickHouse/pull/29692) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). * BorrowedObjectPool condition variable notify fix [#29722](https://github.com/ClickHouse/ClickHouse/pull/29722) ([Maksim Kita](https://github.com/kitaisreal)). * Better exception message for local interactive [#29737](https://github.com/ClickHouse/ClickHouse/pull/29737) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix --stage for clickhouse-local [#29745](https://github.com/ClickHouse/ClickHouse/pull/29745) ([Azat Khuzhin](https://github.com/azat)). @@ -308,7 +308,7 @@ sidebar_label: 2022 * Fix client [#29864](https://github.com/ClickHouse/ClickHouse/pull/29864) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove some more streams. [#29898](https://github.com/ClickHouse/ClickHouse/pull/29898) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Add logging in ZooKeeper client [#29901](https://github.com/ClickHouse/ClickHouse/pull/29901) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix some flacky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix some flaky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). * Grep server log even if it contains binary data [#29903](https://github.com/ClickHouse/ClickHouse/pull/29903) ([Alexander Tokmakov](https://github.com/tavplubix)). * Cosmetic refactoring of server constants. [#29913](https://github.com/ClickHouse/ClickHouse/pull/29913) ([Amos Bird](https://github.com/amosbird)). * Format improvement of AlterQuery [#29916](https://github.com/ClickHouse/ClickHouse/pull/29916) ([flynn](https://github.com/ucasfl)). @@ -465,7 +465,7 @@ sidebar_label: 2022 * Fix docs release [#30933](https://github.com/ClickHouse/ClickHouse/pull/30933) ([alesapin](https://github.com/alesapin)). * Fix style check [#30937](https://github.com/ClickHouse/ClickHouse/pull/30937) ([alesapin](https://github.com/alesapin)). * Fix file progress for clickhouse-local [#30938](https://github.com/ClickHouse/ClickHouse/pull/30938) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix reading from TinyLog [#30941](https://github.com/ClickHouse/ClickHouse/pull/30941) ([Vitaly Baranov](https://github.com/vitlibar)). * Add github to known hosts in docs release [#30947](https://github.com/ClickHouse/ClickHouse/pull/30947) ([alesapin](https://github.com/alesapin)). * Parse json from response in ci checks [#30948](https://github.com/ClickHouse/ClickHouse/pull/30948) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.12.1.9017-prestable.md b/docs/changelogs/v21.12.1.9017-prestable.md index bd84873e67a..f5416664d35 100644 --- a/docs/changelogs/v21.12.1.9017-prestable.md +++ b/docs/changelogs/v21.12.1.9017-prestable.md @@ -220,7 +220,7 @@ sidebar_label: 2022 * Fix test_backward_compatibility [#30950](https://github.com/ClickHouse/ClickHouse/pull/30950) ([Ilya Yatsishin](https://github.com/qoega)). * Add stress test to github actions [#30952](https://github.com/ClickHouse/ClickHouse/pull/30952) ([alesapin](https://github.com/alesapin)). * Try smaller blacklist of non parallel integration tests [#30963](https://github.com/ClickHouse/ClickHouse/pull/30963) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). * Move access-rights source code [#30973](https://github.com/ClickHouse/ClickHouse/pull/30973) ([Vitaly Baranov](https://github.com/vitlibar)). * Set output_format_avro_rows_in_file default to 1 [#30990](https://github.com/ClickHouse/ClickHouse/pull/30990) ([Kruglov Pavel](https://github.com/Avogar)). * Remove remaining usages of Y_IGNORE [#30993](https://github.com/ClickHouse/ClickHouse/pull/30993) ([Yuriy Chernyshov](https://github.com/georgthegreat)). @@ -353,7 +353,7 @@ sidebar_label: 2022 * Support toUInt8/toInt8 for if constant condition optimization. [#31866](https://github.com/ClickHouse/ClickHouse/pull/31866) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Added -no-sanitize=unsigned-integer-overflow build flag [#31881](https://github.com/ClickHouse/ClickHouse/pull/31881) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix typos [#31886](https://github.com/ClickHouse/ClickHouse/pull/31886) ([Anton Popov](https://github.com/CurtizJ)). -* Try to fix flacky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Try to fix flaky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Reduce the files that depend on parser headers [#31896](https://github.com/ClickHouse/ClickHouse/pull/31896) ([Raúl Marín](https://github.com/Algunenano)). * Fix magic_enum for debug helpers (fixes build w/ USE_DEBUG_HELPERS) [#31922](https://github.com/ClickHouse/ClickHouse/pull/31922) ([Azat Khuzhin](https://github.com/azat)). * Remove some trash from build [#31923](https://github.com/ClickHouse/ClickHouse/pull/31923) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -387,7 +387,7 @@ sidebar_label: 2022 * make looping in H3 funcs uniform [#32110](https://github.com/ClickHouse/ClickHouse/pull/32110) ([Bharat Nallan](https://github.com/bharatnc)). * Remove PVS check from master [#32114](https://github.com/ClickHouse/ClickHouse/pull/32114) ([alesapin](https://github.com/alesapin)). * Fix flaky keeper whitelist test [#32115](https://github.com/ClickHouse/ClickHouse/pull/32115) ([alesapin](https://github.com/alesapin)). -* Fix flacky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix flaky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). * Fix data race in `removePartAndEnqueueFetch(...)` [#32119](https://github.com/ClickHouse/ClickHouse/pull/32119) ([Alexander Tokmakov](https://github.com/tavplubix)). * Move fuzzers and unit tests to another group [#32120](https://github.com/ClickHouse/ClickHouse/pull/32120) ([alesapin](https://github.com/alesapin)). * Add a test with 20000 mutations in one query [#32122](https://github.com/ClickHouse/ClickHouse/pull/32122) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). @@ -411,11 +411,11 @@ sidebar_label: 2022 * Add test for [#32186](https://github.com/ClickHouse/ClickHouse/issues/32186) [#32203](https://github.com/ClickHouse/ClickHouse/pull/32203) ([Raúl Marín](https://github.com/Algunenano)). * Fix uncaught exception in DatabaseLazy [#32206](https://github.com/ClickHouse/ClickHouse/pull/32206) ([Alexander Tokmakov](https://github.com/tavplubix)). * Update ASTCreateQuery.cpp [#32208](https://github.com/ClickHouse/ClickHouse/pull/32208) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix jemalloc under osx [#32219](https://github.com/ClickHouse/ClickHouse/pull/32219) ([Azat Khuzhin](https://github.com/azat)). * Add missing timezones to some tests [#32222](https://github.com/ClickHouse/ClickHouse/pull/32222) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix versioning of aggregate functions (fixes performance tests) [#32236](https://github.com/ClickHouse/ClickHouse/pull/32236) ([Azat Khuzhin](https://github.com/azat)). -* Disable window view tests temporarily because still flacky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disable window view tests temporarily because still flaky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix typo in tupleToNameValuePairs doc [#32262](https://github.com/ClickHouse/ClickHouse/pull/32262) ([Vladimir C](https://github.com/vdimir)). * Fix possible Pipeline stuck in case of StrictResize processor. [#32270](https://github.com/ClickHouse/ClickHouse/pull/32270) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix possible crash in DataTypeAggregateFunction [#32287](https://github.com/ClickHouse/ClickHouse/pull/32287) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). diff --git a/docs/changelogs/v21.5.1.6601-prestable.md b/docs/changelogs/v21.5.1.6601-prestable.md index b7dd8ae87c0..69ea9cb8d0e 100644 --- a/docs/changelogs/v21.5.1.6601-prestable.md +++ b/docs/changelogs/v21.5.1.6601-prestable.md @@ -158,7 +158,7 @@ sidebar_label: 2022 * MemoryStorage sync comments and code [#22721](https://github.com/ClickHouse/ClickHouse/pull/22721) ([Maksim Kita](https://github.com/kitaisreal)). * Fix potential segfault on Keeper startup [#22743](https://github.com/ClickHouse/ClickHouse/pull/22743) ([alesapin](https://github.com/alesapin)). * Avoid using harmful function rand() [#22744](https://github.com/ClickHouse/ClickHouse/pull/22744) ([Amos Bird](https://github.com/amosbird)). -* Fix flacky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix flaky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). * add more messages when flushing the logs [#22761](https://github.com/ClickHouse/ClickHouse/pull/22761) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Moved BorrowedObjectPool to common [#22764](https://github.com/ClickHouse/ClickHouse/pull/22764) ([Maksim Kita](https://github.com/kitaisreal)). * Functions ExternalDictionaries standardize exception throw [#22821](https://github.com/ClickHouse/ClickHouse/pull/22821) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v21.6.9.7-stable.md b/docs/changelogs/v21.6.9.7-stable.md index 0a989e4d6b7..533c58badac 100644 --- a/docs/changelogs/v21.6.9.7-stable.md +++ b/docs/changelogs/v21.6.9.7-stable.md @@ -55,7 +55,7 @@ sidebar_label: 2022 * Try fix rabbitmq tests [#26826](https://github.com/ClickHouse/ClickHouse/pull/26826) ([Kseniia Sumarokova](https://github.com/kssenii)). * One more library bridge fix [#26873](https://github.com/ClickHouse/ClickHouse/pull/26873) ([Kseniia Sumarokova](https://github.com/kssenii)). * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.7.9.7-stable.md b/docs/changelogs/v21.7.9.7-stable.md index 7aaab54af6b..684d0e8995e 100644 --- a/docs/changelogs/v21.7.9.7-stable.md +++ b/docs/changelogs/v21.7.9.7-stable.md @@ -35,7 +35,7 @@ sidebar_label: 2022 #### NOT FOR CHANGELOG / INSIGNIFICANT * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.8.1.7409-prestable.md b/docs/changelogs/v21.8.1.7409-prestable.md index cb6ab82b30f..6ef2f1b50d2 100644 --- a/docs/changelogs/v21.8.1.7409-prestable.md +++ b/docs/changelogs/v21.8.1.7409-prestable.md @@ -101,7 +101,7 @@ sidebar_label: 2022 * Separate log files for separate runs in stress test [#25741](https://github.com/ClickHouse/ClickHouse/pull/25741) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix slow performance test [#25742](https://github.com/ClickHouse/ClickHouse/pull/25742) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * DatabaseAtomic EXCHANGE DICTIONARIES fix test [#25753](https://github.com/ClickHouse/ClickHouse/pull/25753) ([Maksim Kita](https://github.com/kitaisreal)). -* Try fix flacky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Try fix flaky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a test for [#13993](https://github.com/ClickHouse/ClickHouse/issues/13993) [#25758](https://github.com/ClickHouse/ClickHouse/pull/25758) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Set follow-fork-mode child for gdb in stress/fasttest/fuzzer [#25769](https://github.com/ClickHouse/ClickHouse/pull/25769) ([Azat Khuzhin](https://github.com/azat)). * Ignore TOO_DEEP_RECURSION server exception during fuzzing [#25770](https://github.com/ClickHouse/ClickHouse/pull/25770) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.5.7-lts.md b/docs/changelogs/v21.8.5.7-lts.md index fa459e093f7..4d0727e362c 100644 --- a/docs/changelogs/v21.8.5.7-lts.md +++ b/docs/changelogs/v21.8.5.7-lts.md @@ -40,7 +40,7 @@ sidebar_label: 2022 * Fix several bugs in ZooKeeper snapshots deserialization [#26127](https://github.com/ClickHouse/ClickHouse/pull/26127) ([alesapin](https://github.com/alesapin)). * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.9.1.8000-prestable.md b/docs/changelogs/v21.9.1.8000-prestable.md index bc921a68693..70ae3697e92 100644 --- a/docs/changelogs/v21.9.1.8000-prestable.md +++ b/docs/changelogs/v21.9.1.8000-prestable.md @@ -346,7 +346,7 @@ sidebar_label: 2022 * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix 01300_client_save_history_when_terminated_long [#27324](https://github.com/ClickHouse/ClickHouse/pull/27324) ([Raúl Marín](https://github.com/Algunenano)). * Try update contrib/zlib-ng [#27327](https://github.com/ClickHouse/ClickHouse/pull/27327) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add and check system.mutations for database filter [#27384](https://github.com/ClickHouse/ClickHouse/pull/27384) ([Azat Khuzhin](https://github.com/azat)). * Correct the key data type used in mapContains [#27423](https://github.com/ClickHouse/ClickHouse/pull/27423) ([Fuwang Hu](https://github.com/fuwhu)). * Fix tests for WithMergeableStateAfterAggregationAndLimit [#27424](https://github.com/ClickHouse/ClickHouse/pull/27424) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.1.1.2542-prestable.md b/docs/changelogs/v22.1.1.2542-prestable.md index cacd13c1e12..3b0422abb11 100644 --- a/docs/changelogs/v22.1.1.2542-prestable.md +++ b/docs/changelogs/v22.1.1.2542-prestable.md @@ -398,7 +398,7 @@ sidebar_label: 2022 * test for [#24410](https://github.com/ClickHouse/ClickHouse/issues/24410) [#33265](https://github.com/ClickHouse/ClickHouse/pull/33265) ([Denny Crane](https://github.com/den-crane)). * Wait for RabbitMQ container to actually start when it was restarted in test on purpose [#33266](https://github.com/ClickHouse/ClickHouse/pull/33266) ([Kseniia Sumarokova](https://github.com/kssenii)). * Mark max_alter_threads as obsolete [#33268](https://github.com/ClickHouse/ClickHouse/pull/33268) ([Denny Crane](https://github.com/den-crane)). -* Fix azure tests flackyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix azure tests flakyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). * Test for [#26920](https://github.com/ClickHouse/ClickHouse/issues/26920) [#33272](https://github.com/ClickHouse/ClickHouse/pull/33272) ([Denny Crane](https://github.com/den-crane)). * Fix test_storage_kafka failures by adjusting retention.ms [#33278](https://github.com/ClickHouse/ClickHouse/pull/33278) ([Azat Khuzhin](https://github.com/azat)). * Disable FunctionConvertFromString::canBeExecutedOnDefaultArguments [#33286](https://github.com/ClickHouse/ClickHouse/pull/33286) ([Vladimir C](https://github.com/vdimir)). @@ -447,7 +447,7 @@ sidebar_label: 2022 * Update mongodb.md [#33585](https://github.com/ClickHouse/ClickHouse/pull/33585) ([Kseniia Sumarokova](https://github.com/kssenii)). * Restore existing static builds links [#33597](https://github.com/ClickHouse/ClickHouse/pull/33597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix pylint for run_check.py [#33600](https://github.com/ClickHouse/ClickHouse/pull/33600) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix flacky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). * Make ZooKeeper client better interpret keeper server connection reject [#33602](https://github.com/ClickHouse/ClickHouse/pull/33602) ([alesapin](https://github.com/alesapin)). * Fix broken workflow dependencies [#33608](https://github.com/ClickHouse/ClickHouse/pull/33608) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Force rebuild images in CI [#33609](https://github.com/ClickHouse/ClickHouse/pull/33609) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). diff --git a/docs/changelogs/v22.4.1.2305-prestable.md b/docs/changelogs/v22.4.1.2305-prestable.md index b277137ca7e..e8304e6505f 100644 --- a/docs/changelogs/v22.4.1.2305-prestable.md +++ b/docs/changelogs/v22.4.1.2305-prestable.md @@ -410,7 +410,7 @@ sidebar_label: 2022 * Fix mongodb test with new cert [#36161](https://github.com/ClickHouse/ClickHouse/pull/36161) ([alesapin](https://github.com/alesapin)). * Some fixes for ReplicatedMergeTree [#36163](https://github.com/ClickHouse/ClickHouse/pull/36163) ([Alexander Tokmakov](https://github.com/tavplubix)). * clickhouse-client: properly cancel query in case of error during formatting data [#36164](https://github.com/ClickHouse/ClickHouse/pull/36164) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). * Revert "Fix possible mutation stuck due to race with DROP_RANGE" [#36190](https://github.com/ClickHouse/ClickHouse/pull/36190) ([Azat Khuzhin](https://github.com/azat)). * Use atomic instead of mutex + condvar in ParallelReadBuffer [#36192](https://github.com/ClickHouse/ClickHouse/pull/36192) ([Kruglov Pavel](https://github.com/Avogar)). * Follow-up to [#36138](https://github.com/ClickHouse/ClickHouse/issues/36138) [#36194](https://github.com/ClickHouse/ClickHouse/pull/36194) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v23.4.1.1943-stable.md b/docs/changelogs/v23.4.1.1943-stable.md index ea16f5856be..34590ba9d37 100644 --- a/docs/changelogs/v23.4.1.1943-stable.md +++ b/docs/changelogs/v23.4.1.1943-stable.md @@ -321,7 +321,7 @@ sidebar_label: 2023 * Add a test for [#38128](https://github.com/ClickHouse/ClickHouse/issues/38128) [#48817](https://github.com/ClickHouse/ClickHouse/pull/48817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove excessive logging [#48826](https://github.com/ClickHouse/ClickHouse/pull/48826) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * remove duplicate indentwith in clang-format [#48834](https://github.com/ClickHouse/ClickHouse/pull/48834) ([cluster](https://github.com/infdahai)). -* Try fix flacky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). +* Try fix flaky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). * fix the race wait loading parts [#48844](https://github.com/ClickHouse/ClickHouse/pull/48844) ([Sema Checherinda](https://github.com/CheSema)). * suppress assert of progress for test_system_replicated_fetches [#48856](https://github.com/ClickHouse/ClickHouse/pull/48856) ([Han Fei](https://github.com/hanfei1991)). * Fix: do not run test_store_cleanup_disk_s3 in parallel [#48863](https://github.com/ClickHouse/ClickHouse/pull/48863) ([Igor Nikonov](https://github.com/devcrafter)). @@ -372,4 +372,3 @@ sidebar_label: 2023 * suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)). * Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)). * Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)). - diff --git a/docs/changelogs/v23.6.1.1524-stable.md b/docs/changelogs/v23.6.1.1524-stable.md index b91c5340789..0de9ab37653 100644 --- a/docs/changelogs/v23.6.1.1524-stable.md +++ b/docs/changelogs/v23.6.1.1524-stable.md @@ -263,7 +263,7 @@ sidebar_label: 2023 * Fix broken labeling for `manual approve` [#51405](https://github.com/ClickHouse/ClickHouse/pull/51405) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix parts lifetime in `MergeTreeTransaction` [#51407](https://github.com/ClickHouse/ClickHouse/pull/51407) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix flaky test test_skip_empty_files [#51409](https://github.com/ClickHouse/ClickHouse/pull/51409) ([Kruglov Pavel](https://github.com/Avogar)). -* fix flacky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). +* fix flaky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). * Update README.md [#51413](https://github.com/ClickHouse/ClickHouse/pull/51413) ([Tyler Hannan](https://github.com/tylerhannan)). * Replace try/catch logic in hasTokenOrNull() by something more lightweight [#51425](https://github.com/ClickHouse/ClickHouse/pull/51425) ([Robert Schulze](https://github.com/rschu1ze)). * Add retries to `tlsv1_3` tests [#51434](https://github.com/ClickHouse/ClickHouse/pull/51434) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index cd77a8c03cf..fb50dfcee85 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -252,7 +252,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" - 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) - 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复flacky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复flaky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 清理重复的链接器标志。 确保链接器不会查找意想不到的符号。 [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([阿莫斯鸟](https://github.com/amosbird)) - 添加 `clickhouse-odbc` 驱动程序进入测试图像。 这允许通过自己的ODBC驱动程序测试ClickHouse与ClickHouse的交互。 [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) - 修复单元测试中的几个错误。 [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([阿利沙平](https://github.com/alesapin)) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index a9307c3be99..7d2c15714e2 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -65,7 +65,7 @@ void SystemLogQueue::push(LogElement&& element) /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. - /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. + /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flaky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; /// Should not log messages under mutex. From bd5022432909bcb633f31617a764ca223ad585d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 20:57:22 +0200 Subject: [PATCH 471/496] Whitespaces --- docs/changelogs/v23.4.1.1943-stable.md | 1 - src/Processors/Chunk.h | 2 +- tests/queries/0_stateless/02099_tsv_raw_format.sh | 7 +++---- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/changelogs/v23.4.1.1943-stable.md b/docs/changelogs/v23.4.1.1943-stable.md index ea16f5856be..6ab6d8b457a 100644 --- a/docs/changelogs/v23.4.1.1943-stable.md +++ b/docs/changelogs/v23.4.1.1943-stable.md @@ -372,4 +372,3 @@ sidebar_label: 2023 * suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)). * Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)). * Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)). - diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 1348966c0d3..f45e2c4619e 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -22,7 +22,7 @@ public: }; -template +template class ChunkInfoCloneable : public ChunkInfo { public: diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh index 16b695e4037..026607ac6d5 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -14,7 +14,7 @@ do echo $format $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 SELECT number, toString(number), toDate(number) FROM numbers(3)" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" - + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099" @@ -49,13 +49,12 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_string_02099" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()"; -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099" -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099" - From a49924b3834d577827279f6278e5df21e3065035 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 13:27:06 +0200 Subject: [PATCH 472/496] tests: avoid leaving leftovers after test_storage_mongodb (fixes flaky chec) CI: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- tests/integration/test_storage_mongodb/test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 1a1a790e8e8..6e2d15b03be 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -410,6 +410,7 @@ def test_no_credentials(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table_2") node.query( "create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')" ) @@ -439,10 +440,13 @@ def test_auth_source(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table_fail") node.query( "create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail") + + node.query("drop table if exists simple_mongo_table_ok") node.query( "create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')" ) From c200f437746ddbd03e5ce7c7ad9d9613015ce929 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 16:00:46 +0200 Subject: [PATCH 473/496] ci: fix stateless runner to correctly gather artifacts when server failed Right now it simply does not work due to "set -e", with it you cannot use "foo=$(false)" since bash will break execution after, rewrite it to a plain shell, with "if". Also use ZSTD everywhere (ugh) Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..5c15c05652b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -299,22 +299,22 @@ stop_logs_replication failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log do - err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes") - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 - err=$( { clickhouse-client --port 29000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client --port 19000 -q "select * from system.$table into outfile '/test_output/$table.1.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi + if ! clickhouse-client --port 29000 -q "select * from system.$table into outfile '/test_output/$table.2.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi fi if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then - err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client --port 29000 -q "select * from system.$table into outfile '/test_output/$table.2.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi fi done From 43cf85ef2841190a5d12a592a65a5181bb110661 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 16:09:48 +0200 Subject: [PATCH 474/496] ci: collect basic issues in stateless tests (dmesg, fatal and similar) Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 9 +++++++++ docker/test/stateless/stress_tests.lib | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5c15c05652b..b352539cc1a 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -3,6 +3,12 @@ # shellcheck disable=SC1091 source /setup_export_logs.sh +# shellcheck source=../stateless/stress_tests.lib +source /stress_tests.lib + +# Avoid overlaps with previous runs +dmesg --clear + # fail on errors, verbose and export all env variables set -e -x -a @@ -420,4 +426,7 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: fi +# Grep logs for sanitizer asserts, crashes and other critical errors +check_logs_for_critical_errors + collect_core_dumps diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 682da1df837..36782101fa7 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -242,7 +242,7 @@ function check_server_start() function check_logs_for_critical_errors() { # Sanitizer asserts - sed -n '/WARNING:.*anitizer/,/^$/p' /var/log/clickhouse-server/stderr.log >> /test_output/tmp + sed -n '/WARNING:.*anitizer/,/^$/p' /var/log/clickhouse-server/stderr*.log >> /test_output/tmp rg -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ && echo -e "Sanitizer assert (in stderr.log)$FAIL$(head_escaped /test_output/tmp)" >> /test_output/test_results.tsv \ || echo -e "No sanitizer asserts$OK" >> /test_output/test_results.tsv From 83be27cdf59ec5886abc23441f5cf92740b04c57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:34:01 +0200 Subject: [PATCH 475/496] Fix test `02833_concurrent_sessions`, Fix test `02835_drop_user_during_session` --- .../0_stateless/02833_concurrent_sessions.sh | 18 ++++++++++++- .../02835_drop_user_during_session.sh | 26 +++++++++++++++---- 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02833_concurrent_sessions.sh b/tests/queries/0_stateless/02833_concurrent_sessions.sh index 846661cfeed..feaff3a38a3 100755 --- a/tests/queries/0_stateless/02833_concurrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrent_sessions.sh @@ -137,7 +137,23 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN ( for user in "${ALL_USERS[@]}"; do ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + + # The client can exit sooner than the server records its disconnection and closes the session. + # When the client disconnects, two processes happen at the same time and are in the race condition: + # - the client application exits and returns control to the shell; + # - the server closes the session and records the logout event to the session log. + # We cannot expect that after the control is returned to the shell, the server records the logout event. + while true + do + [[ 3 -eq $(${CLICKHOUSE_CLIENT} -q " + SELECT COUNT(*) FROM ( + SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' + INTERSECT + SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout' + )") ]] && echo 3 && break; + sleep 0.1 + done + echo "LoginFailure" ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" done diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index 347ebd22f96..c32003a2a11 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -24,7 +24,7 @@ function http_session() ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" } -function http_with_session_id_session() +function http_with_session_id_session() { local user=$1 ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" @@ -104,11 +104,27 @@ wait ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -echo "port_0_sessions:" +echo "port_0_sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" echo "address_0_sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "Corresponding LoginSuccess/Logout" + +# The client can exit sooner than the server records its disconnection and closes the session. +# When the client disconnects, two processes happen at the same time and are in the race condition: +# - the client application exits and returns control to the shell; +# - the server closes the session and records the logout event to the session log. +# We cannot expect that after the control is returned to the shell, the server records the logout event. +while true +do + [[ 9 -eq $(${CLICKHOUSE_CLIENT} -q " + SELECT COUNT(*) FROM ( + SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' + INTERSECT + SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' + )") ]] && echo 9 && break; + sleep 0.1 +done + echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 0bdaa57023ef69e49ab5cf9d54ed2e52c1fd2dae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:53:54 +0200 Subject: [PATCH 476/496] Fix diagnostics in the test script --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..bec8b9cc4d1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -260,7 +260,7 @@ function run_tests() | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e - DURATION=$((START_TIME - SECONDS)) + DURATION=$((SECONDS - START_TIME)) echo "Elapsed ${DURATION} seconds." if [[ $DURATION -ge $TIMEOUT ]] From a573b2926e13ea31c7947b0429b1b5723c7fb938 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:59:01 +0200 Subject: [PATCH 477/496] Fixes for the script --- docker/test/stateless/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..8b9e729970c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -72,8 +72,12 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th remove_keeper_config "latest_logs_cache_size_threshold" "[[:digit:]]\+" fi +export IS_FLAKY_CHECK=0 + # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then + export IS_FLAKY_CHECK=1 + export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 export THREAD_FUZZER_SLEEP_PROBABILITY=0.1 export THREAD_FUZZER_SLEEP_TIME_US_MAX=100000 From 538761b43dbf704d0700548e61b4034ef66c5766 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:02:32 +0200 Subject: [PATCH 478/496] Fix flaky check --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b70dd61a25a..38b0e99760e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,7 +1232,7 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and (args.test_runs > 1): + elif "no-flaky-check" in tags and (1 == int(os.environ.get("IS_FLAKY_CHECK", 0))): return FailureReason.SKIP elif tags: From 270dddc00fd5533ba24914e7d9ae0aebc50a3fd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:22:48 +0200 Subject: [PATCH 479/496] Fix test `02231_bloom_filter_sizing` --- tests/queries/0_stateless/02231_bloom_filter_sizing.reference | 4 ++-- tests/queries/0_stateless/02231_bloom_filter_sizing.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02231_bloom_filter_sizing.reference b/tests/queries/0_stateless/02231_bloom_filter_sizing.reference index bdba311c092..aabadfc92fd 100644 --- a/tests/queries/0_stateless/02231_bloom_filter_sizing.reference +++ b/tests/queries/0_stateless/02231_bloom_filter_sizing.reference @@ -1,6 +1,6 @@ Bloom filter on sort key -10000 +1000 0 Bloom filter on non-sort key -10000 +1000 0 diff --git a/tests/queries/0_stateless/02231_bloom_filter_sizing.sql b/tests/queries/0_stateless/02231_bloom_filter_sizing.sql index 233e3111067..ee896675d64 100644 --- a/tests/queries/0_stateless/02231_bloom_filter_sizing.sql +++ b/tests/queries/0_stateless/02231_bloom_filter_sizing.sql @@ -12,7 +12,7 @@ INSERT INTO bloom_filter_sizing_pk SELECT number % 100 as key, -- 100 unique keys number as value -- whatever -FROM numbers(1000 * 1000); +FROM numbers(100_000); -- -- Merge everything into a single part @@ -40,7 +40,7 @@ SELECT number % 100 as key1, -- 100 unique keys rand() % 100 as key2, -- 100 unique keys number as value -- whatever -FROM numbers(1000 * 1000); +FROM numbers(100_000); -- -- Merge everything into a single part From 5a860fcc3aaf4e77025cd9286da10d0587134c3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:25:56 +0200 Subject: [PATCH 480/496] Update 02099_tsv_raw_format.sh --- tests/queries/0_stateless/02099_tsv_raw_format.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh index 026607ac6d5..a69c96ab613 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 7adeaf9c28016b29453d6be2bca1aa1ef866ecdf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 4 Aug 2024 22:35:07 +0000 Subject: [PATCH 481/496] Automatic style fix --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 38b0e99760e..907d773337a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,7 +1232,9 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and (1 == int(os.environ.get("IS_FLAKY_CHECK", 0))): + elif "no-flaky-check" in tags and ( + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + ): return FailureReason.SKIP elif tags: From 6b1e184e12bed759487d89f54f5ac4f269dffda2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 22:50:46 +0200 Subject: [PATCH 482/496] Print original query for AST formatting check on CI It may be tricky to understand the root cause of the AST formatting issue in case of syntax error, so add one knob to control this - debug_ast_formatting_print_original_query. And CI contains core dumps anyway, so let's enable for CI. P.S. There was concern from @al13n321 that printing original query even in debug build is not a good idea [1], hence a knob for this. [1]: https://github.com/ClickHouse/ClickHouse/pull/63357/files#r1674809348 Signed-off-by: Azat Khuzhin --- src/Interpreters/executeQuery.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ba6fc0f14a0..ce58f7f922c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -796,10 +796,9 @@ static std::tuple executeQueryImpl( catch (const Exception & e) { if (e.code() == ErrorCodes::SYNTAX_ERROR) - /// Don't print the original query text because it may contain sensitive data. throw Exception(ErrorCodes::LOGICAL_ERROR, - "Inconsistent AST formatting: the query:\n{}\ncannot parse.", - formatted1); + "Inconsistent AST formatting: the query:\n{}\ncannot parse query back from {}", + formatted1, std::string_view(begin, end-begin)); else throw; } From 8bca80f4dd08d8ad05db0325b96365d82e6c4076 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 22:35:21 +0200 Subject: [PATCH 483/496] Fix REPLACE modifier formatting (forbid omitting brackets) It is too tricky to verify does brackets required or not, i.e. "SELECT * REPLACE(1/3/3 AS dummy)" will be formatted to "SELECT * REPLACE (1/3)/3 AS dummy" which is already invalid query. So let's simply always print them. Signed-off-by: Azat Khuzhin v2: move the fix into correct place ASTColumnsReplaceTransformer::formatImpl() instead of ASTColumnsReplaceTransformer::Replacement::formatImpl() --- src/Parsers/ASTColumnsTransformers.cpp | 8 ++------ .../03220_replace_formatting.reference | 16 ++++++++++++++++ .../0_stateless/03220_replace_formatting.sh | 14 ++++++++++++++ 3 files changed, 32 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03220_replace_formatting.reference create mode 100755 tests/queries/0_stateless/03220_replace_formatting.sh diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 2a61892f8cc..332ebca3bdb 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -323,9 +323,7 @@ void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, F { settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : ""); - if (children.size() > 1) - settings.ostr << "("; - + settings.ostr << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { if (it != children.begin()) @@ -333,9 +331,7 @@ void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, F (*it)->formatImpl(settings, state, frame); } - - if (children.size() > 1) - settings.ostr << ")"; + settings.ostr << ")"; } void ASTColumnsReplaceTransformer::appendColumnName(WriteBuffer & ostr) const diff --git a/tests/queries/0_stateless/03220_replace_formatting.reference b/tests/queries/0_stateless/03220_replace_formatting.reference new file mode 100644 index 00000000000..cbcd63839b1 --- /dev/null +++ b/tests/queries/0_stateless/03220_replace_formatting.reference @@ -0,0 +1,16 @@ +SELECT * REPLACE ((1 / 3) / 3 AS dummy) +SELECT * REPLACE ((1 / 3) / 3 AS dummy) +SELECT * REPLACE STRICT (1 AS id, 2 AS value) +FROM +( + SELECT + 0 AS id, + 1 AS value +) +SELECT * REPLACE STRICT (1 AS id, 2 AS value) +FROM +( + SELECT + 0 AS id, + 1 AS value +) diff --git a/tests/queries/0_stateless/03220_replace_formatting.sh b/tests/queries/0_stateless/03220_replace_formatting.sh new file mode 100755 index 00000000000..1c11ed6da8d --- /dev/null +++ b/tests/queries/0_stateless/03220_replace_formatting.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +q=$($CLICKHOUSE_FORMAT <<<"SELECT * REPLACE(1/3/3 AS dummy)") +echo "$q" +$CLICKHOUSE_FORMAT <<<"$q" + +# multiple columns +q=$($CLICKHOUSE_FORMAT <<<"SELECT * REPLACE STRICT (1 AS id, 2 AS value) FROM (SELECT 0 id, 1 value)") +echo "$q" +$CLICKHOUSE_FORMAT <<<"$q" From 2a7ad3a1f979708fd152e364296c505db8926aba Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 23:21:19 +0200 Subject: [PATCH 484/496] Update test references for new REPLACE modifier syntax Signed-off-by: Azat Khuzhin --- .../01913_fix_column_transformer_replace_format.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference b/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference index 33be11c07d5..6fabd33c804 100644 --- a/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference +++ b/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference @@ -1 +1 @@ -CREATE VIEW default.my_view\n(\n `Id` UInt32,\n `Object.Key` Array(UInt16),\n `Object.Value` Array(String)\n)\nAS SELECT * REPLACE arrayMap(x -> (x + 1), `Object.Key`) AS `Object.Key`\nFROM default.my_table +CREATE VIEW default.my_view\n(\n `Id` UInt32,\n `Object.Key` Array(UInt16),\n `Object.Value` Array(String)\n)\nAS SELECT * REPLACE (arrayMap(x -> (x + 1), `Object.Key`) AS `Object.Key`)\nFROM default.my_table From b47f8a733f926c0f52a5837414bcd6ecfece9089 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 08:40:35 +0200 Subject: [PATCH 485/496] ci: fix basic errors collecting after stateless tests Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b352539cc1a..c582d3a982b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -393,6 +393,8 @@ do | zstd --threads=0 > "/test_output/trace-log-$trace_type-flamegraph.tsv.zst" ||: done +# Grep logs for sanitizer asserts, crashes and other critical errors +check_logs_for_critical_errors # Compressed (FIXME: remove once only github actions will be left) rm /var/log/clickhouse-server/clickhouse-server.log @@ -426,7 +428,4 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: fi -# Grep logs for sanitizer asserts, crashes and other critical errors -check_logs_for_critical_errors - collect_core_dumps From cdbc4f357324ad0b41d46b6e54475ac2cebdc630 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 08:45:20 +0200 Subject: [PATCH 486/496] ci: fail the test if the entrypoint script failed Signed-off-by: Azat Khuzhin --- tests/ci/functional_test_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 66db082677f..52970404d2d 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -166,6 +166,7 @@ def _get_statless_tests_to_run(pr_info: PRInfo) -> List[str]: def process_results( + ret_code: int, result_directory: Path, server_log_path: Path, ) -> Tuple[StatusType, str, TestResults, List[Path]]: @@ -192,6 +193,9 @@ def process_results( logging.info("Files in result folder %s", os.listdir(result_directory)) return ERROR, "Invalid check_status.tsv", test_results, additional_files state, description = status[0][0], status[0][1] + if ret_code != 0: + state = ERROR + description += " (but script exited with an error)" try: results_path = result_directory / "test_results.tsv" @@ -339,7 +343,7 @@ def main(): ci_logs_credentials.clean_ci_logs_from_credentials(run_log_path) state, description, test_results, additional_logs = process_results( - result_path, server_log_path + retcode, result_path, server_log_path ) else: print( From 9ce55b69b49ac8426ed0b3db16b95964e3c4db4d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 11:22:55 +0200 Subject: [PATCH 487/496] Fix possible CANNOT_READ_ALL_DATA during server startup in performance tests CI [1]: 2024.08.04 22:09:11.646800 [ 1052 ] {} Application: Code: 33. DB::Exception: Cannot read all data. Bytes read: 0. Bytes expected: 4.: While checking access for disk backups. (CANNOT_READ_ALL_DATA), Stack trace (when copying this message, always include the lines below): [1]: https://s3.amazonaws.com/clickhouse-test-reports/64955/6702acf6f2e4a0ee9697066e38006631fc7f69df/performance_comparison__aarch64__[2_4].html Signed-off-by: Azat Khuzhin --- tests/performance/scripts/compare.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/performance/scripts/compare.sh b/tests/performance/scripts/compare.sh index cb56ab6c5bf..da7bbf77a28 100755 --- a/tests/performance/scripts/compare.sh +++ b/tests/performance/scripts/compare.sh @@ -71,6 +71,8 @@ function configure { # Use the new config for both servers, so that we can change it in a PR. rm right/config/config.d/text_log.xml ||: + # backups disk uses absolute path, and this overlaps between servers, that could lead to errors + rm right/config/config.d/backups.xml ||: cp -rv right/config left ||: # Start a temporary server to rename the tables From a499cd25c7e12c05f2f8fa3fe546715c751ad88d Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:31:41 +0000 Subject: [PATCH 488/496] Fix for integers --- src/Functions/if.cpp | 16 ++++++---------- ...3215_varian_as_common_type_integers.reference | 8 ++++++++ .../03215_varian_as_common_type_integers.sql | 8 ++++++++ 3 files changed, 22 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_integers.reference create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_integers.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 64da6e95a43..8829b3c4ff1 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -632,11 +632,6 @@ private: ColumnPtr executeTuple(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - /// For different Tuples the result type can be Variant with this Tuples if use_variant_as_common_type=1. - /// In this case we should use generic implementation. - if (!isTuple(result_type)) - return nullptr; - /// Calculate function for each corresponding elements of tuples. const ColumnWithTypeAndName & arg1 = arguments[1]; @@ -682,11 +677,6 @@ private: ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - /// For different Maps the result type can be Variant with this Maps if use_variant_as_common_type=1. - /// In this case we should use generic implementation. - if (!isMap(result_type)) - return nullptr; - auto extract_kv_from_map = [](const ColumnMap * map) { const ColumnTuple & tuple = map->getNestedData(); @@ -1243,6 +1233,12 @@ public: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " "Must be ColumnUInt8 or ColumnConstUInt8.", arg_cond.column->getName(), getName()); + /// If result is Variant, always use generic implementation. + /// Using typed implementations may lead to incorrect result column type when + /// resulting Variant is created by use_variant_when_no_common_type. + if (isVariant(result_type)) + return executeGeneric(cond_col, arguments, input_rows_count, use_variant_when_no_common_type); + auto call = [&](const auto & types) -> bool { using Types = std::decay_t; diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference b/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference new file mode 100644 index 00000000000..c5edc9e9963 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference @@ -0,0 +1,8 @@ +0 Variant(Int64, UInt64) +1 Variant(Int64, UInt64) +0 Variant(Int32, UInt64) +1 Variant(Int32, UInt64) +0 Variant(Int16, UInt64) +1 Variant(Int16, UInt64) +0 Variant(Int8, UInt64) +1 Variant(Int8, UInt64) diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql b/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql new file mode 100644 index 00000000000..dcc69735534 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql @@ -0,0 +1,8 @@ +set use_variant_as_common_type = 1; +set allow_experimental_variant_type = 1; + +SELECT if(number % 2, number::Int64, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int32, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int16, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int8, number::UInt64) as res, toTypeName(res) FROM numbers(2); + From 2b369cccdd58902e9da3fd3947e5cb5759ba2881 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:35:56 +0000 Subject: [PATCH 489/496] Reduce table size in 03037_dynamic_merges_2_vertical_wide_merge_tree test --- ...3037_dynamic_merges_2_vertical_wide_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_vertical_wide_merge_tree.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index 104d6018e41..dd643f8dffd 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); From cec8a5d52b83f0c1cdcaed833aec9bf79941b2a8 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:42:22 +0000 Subject: [PATCH 490/496] Reduce table size in similar tests --- ...dynamic_merges_2_horizontal_compact_merge_tree.reference | 6 +++--- ...03037_dynamic_merges_2_horizontal_compact_merge_tree.sql | 6 +++--- ...37_dynamic_merges_2_horizontal_wide_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_horizontal_wide_merge_tree.sql | 6 +++--- ...7_dynamic_merges_2_vertical_compact_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_vertical_compact_merge_tree.sql | 6 +++--- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index e133ac3001f..fa64ed2f8fd 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index d527081b763..4b8a036f166 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index ebccfb77922..a4e67de76db 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); From 0a7a67b8e0e05bc9476d4f9dd38747bf61b6bb8e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:57:13 +0000 Subject: [PATCH 491/496] Disable 03038_nested_dynamic_merges* under sanitizers because it's too slow --- ...sted_dynamic_merges_compact_horizontal.sql | 2 +- ...nested_dynamic_merges_compact_vertical.sql | 2 +- ...8_nested_dynamic_merges_small.reference.j2 | 84 +++++++++++++++++++ .../03038_nested_dynamic_merges_small.sql.j2 | 35 ++++++++ ..._nested_dynamic_merges_wide_horizontal.sql | 2 +- ...38_nested_dynamic_merges_wide_vertical.sql | 2 +- 6 files changed, 123 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 1d5c63dcdf1..81888946681 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index 2bffe35c577..ba58ca471a2 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 new file mode 100644 index 00000000000..ae07c164074 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 @@ -0,0 +1,84 @@ +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 new file mode 100644 index 00000000000..7828c2af49c --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 @@ -0,0 +1,35 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test; + +{% for engine in ['MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', + 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1', + 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', + 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1'] -%} + +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; + +system stop merges test; +insert into test select number, number from numbers(10); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index fb686091ebb..a53c5b0b2a5 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index ed195452d56..4256b010ec0 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From 3802b1ed6c7174b0d95bf1c89d339187fe6dc69d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 12:42:57 +0200 Subject: [PATCH 492/496] Update comment --- src/Interpreters/Cache/FileCache.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 4c17afb79be..aff4e48d01d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -326,6 +326,8 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// ^ ^ /// right offset aligned_right_offset /// [_________] <-- last cached file segment, e.g. we have uncovered suffix of the requested range + /// ^ + /// last_file_segment_right_offset /// [________________] /// size /// [____________________________________] @@ -335,8 +337,9 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// and get something like this: /// /// [________________________] - /// ^ ^ - /// right_offset right_offset + max_file_segment_size + /// ^ ^ + /// | last_file_segment_right_offset + max_file_segment_size + /// last_file_segment_right_offset /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. /// Because its left offset would be bigger than right_offset. /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. From 6a0c0e7b1d4922b616d706820d9000ffe8040d63 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 12:48:21 +0200 Subject: [PATCH 493/496] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index aff4e48d01d..ed91e41db17 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -340,7 +340,7 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// ^ ^ /// | last_file_segment_right_offset + max_file_segment_size /// last_file_segment_right_offset - /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. + /// e.g. there is no need to create sub-segment for range (last_file_segment_right_offset + max_file_segment_size, aligned_right_offset]. /// Because its left offset would be bigger than right_offset. /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. From 3244002cae58da99c8b088888376c0fda7f3f1f4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 13:22:10 +0200 Subject: [PATCH 494/496] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ed91e41db17..1a15efa7cf8 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -560,7 +560,7 @@ FileCache::getOrSet( FileSegment::Range initial_range(offset, offset + size - 1); /// result_range is initial range, which will be adjusted according to - /// 1. aligned offset, alighed_end_offset + /// 1. aligned_offset, aligned_end_offset /// 2. max_file_segments_limit FileSegment::Range result_range = initial_range; From c21b97672c86a18b693c8c60335271248c279fdd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:36:54 +0200 Subject: [PATCH 495/496] Fix trailing whitespace --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1a15efa7cf8..217ae614c22 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -337,7 +337,7 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// and get something like this: /// /// [________________________] - /// ^ ^ + /// ^ ^ /// | last_file_segment_right_offset + max_file_segment_size /// last_file_segment_right_offset /// e.g. there is no need to create sub-segment for range (last_file_segment_right_offset + max_file_segment_size, aligned_right_offset]. From e4c71aec4dfa40d7e9b75c626b0229a33c59f315 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 5 Aug 2024 16:12:25 +0000 Subject: [PATCH 496/496] Update version_date.tsv and changelogs after v24.5.5.78-stable --- docs/changelogs/v24.5.5.78-stable.md | 55 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 56 insertions(+) create mode 100644 docs/changelogs/v24.5.5.78-stable.md diff --git a/docs/changelogs/v24.5.5.78-stable.md b/docs/changelogs/v24.5.5.78-stable.md new file mode 100644 index 00000000000..415ea165101 --- /dev/null +++ b/docs/changelogs/v24.5.5.78-stable.md @@ -0,0 +1,55 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.5.78-stable (0138248cb62) FIXME as compared to v24.5.4.49-stable (63b760955a0) + +#### Improvement +* Backported in [#66768](https://github.com/ClickHouse/ClickHouse/issues/66768): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#66884](https://github.com/ClickHouse/ClickHouse/issues/66884): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66691](https://github.com/ClickHouse/ClickHouse/issues/66691): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67814](https://github.com/ClickHouse/ClickHouse/issues/67814): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67501](https://github.com/ClickHouse/ClickHouse/issues/67501): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67850](https://github.com/ClickHouse/ClickHouse/issues/67850): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65350](https://github.com/ClickHouse/ClickHouse/issues/65350): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65621](https://github.com/ClickHouse/ClickHouse/issues/65621): Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65933](https://github.com/ClickHouse/ClickHouse/issues/65933): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#66301](https://github.com/ClickHouse/ClickHouse/issues/66301): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66328](https://github.com/ClickHouse/ClickHouse/issues/66328): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66155](https://github.com/ClickHouse/ClickHouse/issues/66155): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66454](https://github.com/ClickHouse/ClickHouse/issues/66454): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66226](https://github.com/ClickHouse/ClickHouse/issues/66226): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66680](https://github.com/ClickHouse/ClickHouse/issues/66680): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66604](https://github.com/ClickHouse/ClickHouse/issues/66604): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66360](https://github.com/ClickHouse/ClickHouse/issues/66360): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66972](https://github.com/ClickHouse/ClickHouse/issues/66972): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66969](https://github.com/ClickHouse/ClickHouse/issues/66969): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66720](https://github.com/ClickHouse/ClickHouse/issues/66720): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66951](https://github.com/ClickHouse/ClickHouse/issues/66951): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66757](https://github.com/ClickHouse/ClickHouse/issues/66757): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66948](https://github.com/ClickHouse/ClickHouse/issues/66948): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67633](https://github.com/ClickHouse/ClickHouse/issues/67633): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67481](https://github.com/ClickHouse/ClickHouse/issues/67481): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67197](https://github.com/ClickHouse/ClickHouse/issues/67197): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67379](https://github.com/ClickHouse/ClickHouse/issues/67379): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67576](https://github.com/ClickHouse/ClickHouse/issues/67576): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66387](https://github.com/ClickHouse/ClickHouse/issues/66387): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66426](https://github.com/ClickHouse/ClickHouse/issues/66426): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66544](https://github.com/ClickHouse/ClickHouse/issues/66544): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66859](https://github.com/ClickHouse/ClickHouse/issues/66859): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66875](https://github.com/ClickHouse/ClickHouse/issues/66875): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67059](https://github.com/ClickHouse/ClickHouse/issues/67059): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66945](https://github.com/ClickHouse/ClickHouse/issues/66945): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67252](https://github.com/ClickHouse/ClickHouse/issues/67252): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67412](https://github.com/ClickHouse/ClickHouse/issues/67412): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cb6b8f588da..75c10fa67b8 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.5.78-stable 2024-08-05 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13