From 852318449f2ab8e0bfaeca9904b90936d751d177 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 20:17:43 +0200 Subject: [PATCH 001/217] 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/217] 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/217] 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 b125f8166f32648572c2ed0d540ded56a97ac628 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 19 Jun 2024 16:41:50 +0100 Subject: [PATCH 004/217] 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 005/217] 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 006/217] 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 007/217] 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 008/217] 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 009/217] 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 5d16ba57aa84ef82ccf7e34a4635ad1d14e7859d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Jul 2024 14:46:49 +0000 Subject: [PATCH 010/217] 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 c6b558c7915b070167649d4e88eafb2613570bd3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 4 Jul 2024 22:30:18 +0200 Subject: [PATCH 011/217] 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 012/217] 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 77272c925dc15acc5fdd0260a1c0aab35b1df3c3 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 15:10:39 +0800 Subject: [PATCH 013/217] 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 014/217] 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 015/217] 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 083e4b17db62121d6905c35480c3a462dc26477b Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 09:34:52 +0800 Subject: [PATCH 016/217] trigger CI From b8fbfd227fb60e0f244bda716ef5a9bb89376986 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 15:41:08 +0800 Subject: [PATCH 017/217] 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 018/217] 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 019/217] 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 020/217] 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 021/217] 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 022/217] 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 48e7708d7bcf575123ea20cee9455e0a4cf26791 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 19 Jul 2024 10:29:13 +0800 Subject: [PATCH 023/217] 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 3f66b39a18a7bf271a9a9f97dfc075866e2409eb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 14:30:27 +0200 Subject: [PATCH 024/217] 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 025/217] 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 026/217] 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 8d7471f8bd2e0c6dc242231c4358448787e6c56f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 20 Jul 2024 00:03:40 +0200 Subject: [PATCH 027/217] 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 028/217] 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 029/217] 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 5fa2db8e4828b004ac10d625df62efcc8711dc98 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 16:59:17 +0100 Subject: [PATCH 030/217] 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 031/217] 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 7f5c58f599d34f690c4a04e4223a2f86a433d0e9 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 20:58:56 +0100 Subject: [PATCH 032/217] 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 033/217] 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 932033fca9bdacbfdb544fac5389e03fa7732eeb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 10:55:45 +0100 Subject: [PATCH 034/217] 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 035/217] 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 beb506a5b8179f5c88a6f5fc90d62b8e74bf0d35 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 12:37:05 +0200 Subject: [PATCH 036/217] 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 bd721950b0401f94be652c11015bd1985c283f3a Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 16:24:17 +0200 Subject: [PATCH 037/217] 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 321766d0b8161a794f11835b4650d30b3723835b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 25 Jul 2024 22:51:14 +0000 Subject: [PATCH 038/217] 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 28e991708be1facd87c3760f7929cd5ddc299805 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 13:45:57 +0200 Subject: [PATCH 039/217] 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 ff5cd2051fc8bfd609a9040ffba02697283e69af Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 17:10:39 +0200 Subject: [PATCH 040/217] 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 109bd52296550f506b14134b934c6eb8fd3ec094 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 26 Jul 2024 23:05:56 +0100 Subject: [PATCH 041/217] impl --- cmake/sanitize.cmake | 2 +- contrib/libprotobuf-mutator | 2 +- src/AggregateFunctions/fuzzers/CMakeLists.txt | 2 +- .../aggregate_function_state_deserialization_fuzzer.cpp | 1 + src/Common/MemoryTracker.cpp | 3 +++ src/Core/SettingsFields.cpp | 3 +++ src/Core/fuzzers/CMakeLists.txt | 2 +- src/DataTypes/fuzzers/CMakeLists.txt | 2 +- src/Formats/fuzzers/CMakeLists.txt | 2 +- src/Functions/DateTimeTransforms.h | 3 +++ src/Functions/parseReadableSize.cpp | 3 +++ src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 3 ++- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- src/Storages/fuzzers/CMakeLists.txt | 2 +- src/Storages/fuzzers/columns_description_fuzzer.cpp | 2 ++ 15 files changed, 25 insertions(+), 9 deletions(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 08716c1196b..3f111bc65c4 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -57,7 +57,7 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") - set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") + set(COVERAGE_FLAGS "SHELL:-fprofile-instr-generate -fcoverage-mapping") endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..1f95f808306 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt index 907a275b4b3..3ff7c86a00c 100644 --- a/src/AggregateFunctions/fuzzers/CMakeLists.txt +++ b/src/AggregateFunctions/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions clickhouse_functions) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 425364efb9c..31fc93e4288 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -12,6 +12,7 @@ #include +#include #include #include diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index ac412684cf7..7c0115467c6 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -86,7 +86,10 @@ inline std::string_view toDescription(OvercommitResult result) bool shouldTrackAllocation(Float64 probability, void * ptr) { +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" return intHash64(uintptr_t(ptr)) < std::numeric_limits::max() * probability; +#pragma clang diagnostic pop } } diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 7d094e2a107..68304a29a23 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -271,9 +271,12 @@ namespace if (d != 0.0 && !std::isnormal(d)) throw Exception( ErrorCodes::CANNOT_PARSE_NUMBER, "A setting's value in seconds must be a normal floating point number or zero. Got {}", d); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" if (d * 1000000 > std::numeric_limits::max() || d * 1000000 < std::numeric_limits::min()) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Cannot convert seconds to microseconds: the setting's value in seconds is too big: {}", d); +#pragma clang diagnostic pop return static_cast(d * 1000000); } diff --git a/src/Core/fuzzers/CMakeLists.txt b/src/Core/fuzzers/CMakeLists.txt index 51db6fa0b53..3c5c0eed4e6 100644 --- a/src/Core/fuzzers/CMakeLists.txt +++ b/src/Core/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) -target_link_libraries (names_and_types_fuzzer PRIVATE dbms) +target_link_libraries (names_and_types_fuzzer PRIVATE dbms clickhouse_functions) diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 939bf5f5e3f..bc640358673 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(data_type_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(data_type_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions clickhouse_functions) diff --git a/src/Formats/fuzzers/CMakeLists.txt b/src/Formats/fuzzers/CMakeLists.txt index 38009aeec1d..99e57bfbca3 100644 --- a/src/Formats/fuzzers/CMakeLists.txt +++ b/src/Formats/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS}) -target_link_libraries(format_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(format_fuzzer PRIVATE dbms clickhouse_aggregate_functions clickhouse_functions) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 5f745f3ccad..9d9acd2483b 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -2147,7 +2147,10 @@ struct Transformer if constexpr (std::is_same_v || std::is_same_v) { +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; +#pragma clang diagnostic pop if (!is_valid_input) { if constexpr (std::is_same_v) diff --git a/src/Functions/parseReadableSize.cpp b/src/Functions/parseReadableSize.cpp index 1abcf7f164f..4f6afb939a5 100644 --- a/src/Functions/parseReadableSize.cpp +++ b/src/Functions/parseReadableSize.cpp @@ -217,7 +217,10 @@ private: } Float64 num_bytes_with_decimals = base * iter->second; +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" if (num_bytes_with_decimals > std::numeric_limits::max()) +#pragma clang diagnostic pop { throw Exception( ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 9310d7d59f7..09af67c337c 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -27,7 +27,8 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) DB::ParserQueryWithOutput parser(input.data() + input.size()); try { - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + DB::ASTPtr ast + = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index 854885ad33b..bab8db5671d 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -14,7 +14,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); const UInt64 max_ast_depth = 1000; ast->checkDepth(max_ast_depth); diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index 719b9b77cd9..e36fccec8df 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms) clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) -target_link_libraries (columns_description_fuzzer PRIVATE dbms) +target_link_libraries (columns_description_fuzzer PRIVATE dbms clickhouse_functions) diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index b703a1e7051..469bfe2fb38 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -1,5 +1,7 @@ #include +#include + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { From f5576867748c8941cfc13360cc28526f2703e575 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 27 Jul 2024 21:24:13 +0100 Subject: [PATCH 042/217] cherry on top of this cake --- cmake/sanitize.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 3f111bc65c4..d8d211b9329 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -58,6 +58,7 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") set(COVERAGE_FLAGS "SHELL:-fprofile-instr-generate -fcoverage-mapping") + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) From 13c93cc90cc35e33cad71095fa0300c5f4f1753e Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sun, 28 Jul 2024 10:29:52 +0000 Subject: [PATCH 043/217] fix hung query when using empty tuple as lhs of function IN Signed-off-by: Duc Canh Le --- src/Analyzer/SetUtils.cpp | 6 ++++++ .../0_stateless/03210_empty_tuple_lhs_of_in.reference | 0 tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.sql | 1 + 3 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.reference create mode 100644 tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.sql diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0ecb3545225..e7d1a22f45a 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -43,6 +43,12 @@ size_t getCompoundTypeDepth(const IDataType & type) const auto & tuple_elements = assert_cast(*current_type).getElements(); if (!tuple_elements.empty()) current_type = tuple_elements.at(0).get(); + else + { + /// Special case: tuple with no element - tuple(). In this case, what's the compound type depth? + /// I'm not certain about the theoretical answer, but from experiment, 1 is the most reasonable choice. + return 1; + } ++result; } diff --git a/tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.reference b/tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.sql b/tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.sql new file mode 100644 index 00000000000..0fc5f072da5 --- /dev/null +++ b/tests/queries/0_stateless/03210_empty_tuple_lhs_of_in.sql @@ -0,0 +1 @@ +SELECT tuple() IN tuple(1) SETTINGS allow_experimental_map_type = 1; -- { serverError INCORRECT_ELEMENT_OF_SET } From e6c708124553b67eb4f18b6e39c96a24ad1c2dbb Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Mon, 29 Jul 2024 12:44:33 +0200 Subject: [PATCH 044/217] 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 56135d18cb40506fc500b497068121bc96e3a9ee Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 11:23:01 +0000 Subject: [PATCH 045/217] Remove duplicated tests --- .../03036_dynamic_read_subcolumns.lib | 35 ------------------- .../03036_dynamic_read_subcolumns_1.reference | 19 ---------- .../03036_dynamic_read_subcolumns_1.sh | 21 ----------- .../03036_dynamic_read_subcolumns_2.reference | 19 ---------- .../03036_dynamic_read_subcolumns_2.sh | 21 ----------- .../03036_dynamic_read_subcolumns_3.reference | 19 ---------- .../03036_dynamic_read_subcolumns_3.sh | 21 ----------- 7 files changed, 155 deletions(-) delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib delete mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh delete mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.reference delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh delete mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.reference delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib deleted file mode 100755 index 4914051db82..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib +++ /dev/null @@ -1,35 +0,0 @@ -#!/usr/bin/env bash - -function test() -{ - echo "test" - $CH_CLIENT -q "insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000" - $CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000" - $CH_CLIENT -q "insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000" - $CH_CLIENT -q "insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000" - $CH_CLIENT -q "insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000" - $CH_CLIENT -q "insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(100000, 100000) settings min_insert_block_size_rows=50000" - - $CH_CLIENT -q "select distinct dynamicType(d) as type from test order by type" - $CH_CLIENT -q "select count() from test where dynamicType(d) == 'UInt64'" - $CH_CLIENT -q "select count() from test where d.UInt64 is not NULL" - $CH_CLIENT -q "select count() from test where dynamicType(d) == 'String'" - $CH_CLIENT -q "select count() from test where d.String is not NULL" - $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Date'" - $CH_CLIENT -q "select count() from test where d.Date is not NULL" - $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'" - $CH_CLIENT -q "select count() from test where not empty(d.\`Array(Variant(String, UInt64))\`)" - $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'" - $CH_CLIENT -q "select count() from test where not empty(d.\`Array(Array(Dynamic))\`)" - $CH_CLIENT -q "select count() from test where d is NULL" - $CH_CLIENT -q "select count() from test where not empty(d.\`Tuple(a Array(Dynamic))\`.a.String)" - - $CH_CLIENT -q "select d, d.UInt64, d.String, d.\`Array(Variant(String, UInt64))\` from test format Null" - $CH_CLIENT -q "select d.UInt64, d.String, d.\`Array(Variant(String, UInt64))\` from test format Null" - $CH_CLIENT -q "select d.Int8, d.Date, d.\`Array(String)\` from test format Null" - $CH_CLIENT -q "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 format Null" - $CH_CLIENT -q "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 format Null" - $CH_CLIENT -q "select d, d.\`Tuple(a UInt64, b String)\`.a, d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64, d.\`Array(Variant(String, UInt64))\`.UInt64 from test format Null" - $CH_CLIENT -q "select d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64, d.\`Array(Dynamic)\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64 from test format Null" - $CH_CLIENT -q "select d.\`Array(Array(Dynamic))\`.size1, d.\`Array(Array(Dynamic))\`.UInt64, d.\`Array(Array(Dynamic))\`.\`Map(String, Tuple(a UInt64))\`.values.a from test format Null" -} diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference deleted file mode 100644 index 0d51ecfac3b..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference +++ /dev/null @@ -1,19 +0,0 @@ -Memory -test -Array(Array(Dynamic)) -Array(Variant(String, UInt64)) -None -String -UInt64 -200000 -200000 -200000 -200000 -0 -0 -200000 -200000 -100000 -100000 -200000 -0 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 aabba731816..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= -# 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.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.reference deleted file mode 100644 index 099b7574566..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.reference +++ /dev/null @@ -1,19 +0,0 @@ -MergeTree compact -test -Array(Array(Dynamic)) -Array(Variant(String, UInt64)) -None -String -UInt64 -200000 -200000 -200000 -200000 -0 -0 -200000 -200000 -100000 -100000 -200000 -0 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 872f4c20a98..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= -# 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.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.reference deleted file mode 100644 index 35db4a22b4c..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.reference +++ /dev/null @@ -1,19 +0,0 @@ -MergeTree wide -test -Array(Array(Dynamic)) -Array(Variant(String, UInt64)) -None -String -UInt64 -200000 -200000 -200000 -200000 -0 -0 -200000 -200000 -100000 -100000 -200000 -0 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 96276c96add..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= -# 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 c48770fdf6f769c5f4678db32ca27245b9fa3158 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Mon, 29 Jul 2024 16:46:38 +0200 Subject: [PATCH 046/217] 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 59c517dd91e996ffadd6d4fe51f8f0cf247467c0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 16:08:38 +0100 Subject: [PATCH 047/217] fix format_fuzzer --- src/Formats/fuzzers/format_fuzzer.cpp | 47 ++++++++++++--------------- 1 file changed, 21 insertions(+), 26 deletions(-) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 46661e4828c..408e7218221 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -20,37 +20,32 @@ #include +using namespace DB; + + +ContextMutablePtr context; + +extern "C" int LLVMFuzzerInitialize(int *, char ***) +{ + if (context) + return true; + + SharedContextHolder shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + + MainThreadStatus::getInstance(); + + registerAggregateFunctions(); + registerFormats(); + + return 0; +} extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try { - using namespace DB; - - static SharedContextHolder shared_context; - static ContextMutablePtr context; - - auto initialize = [&]() mutable - { - if (context) - return true; - - shared_context = Context::createShared(); - context = Context::createGlobal(shared_context.get()); - context->makeGlobalContext(); - context->setApplicationType(Context::ApplicationType::LOCAL); - - MainThreadStatus::getInstance(); - - registerAggregateFunctions(); - registerFormats(); - - return true; - }; - - static bool initialized = initialize(); - (void) initialized; - total_memory_tracker.resetCounters(); total_memory_tracker.setHardLimit(1_GiB); CurrentThread::get().memory_tracker.resetCounters(); From e1677bc1b60be3de954c6305b665732e1d4fe062 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 17:13:05 +0100 Subject: [PATCH 048/217] fix rest --- ..._function_state_deserialization_fuzzer.cpp | 45 ++++++++-------- .../data_type_deserialization_fuzzer.cpp | 43 +++++++-------- .../fuzzers/execute_query_fuzzer.cpp | 54 +++++++++---------- 3 files changed, 65 insertions(+), 77 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 31fc93e4288..87979259c9e 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -17,34 +17,31 @@ #include +using namespace DB; + + +ContextMutablePtr context; + +extern "C" int LLVMFuzzerInitialize(int *, char ***) +{ + if (context) + return true; + + SharedContextHolder shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + + MainThreadStatus::getInstance(); + + registerAggregateFunctions(); + + return 0; +} + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try { - using namespace DB; - - static SharedContextHolder shared_context; - static ContextMutablePtr context; - - auto initialize = [&]() mutable - { - if (context) - return true; - - shared_context = Context::createShared(); - context = Context::createGlobal(shared_context.get()); - context->makeGlobalContext(); - context->setApplicationType(Context::ApplicationType::LOCAL); - - MainThreadStatus::getInstance(); - - registerAggregateFunctions(); - return true; - }; - - static bool initialized = initialize(); - (void) initialized; - total_memory_tracker.resetCounters(); total_memory_tracker.setHardLimit(1_GiB); CurrentThread::get().memory_tracker.resetCounters(); diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 033a6ea8a4a..f9a733647e1 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -12,35 +12,30 @@ #include +using namespace DB; + + +ContextMutablePtr context; + +extern "C" int LLVMFuzzerInitialize(int *, char ***) +{ + if (context) + return true; + + SharedContextHolder shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + + MainThreadStatus::getInstance(); + + registerAggregateFunctions(); + return 0; +} extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try { - using namespace DB; - - static SharedContextHolder shared_context; - static ContextMutablePtr context; - - auto initialize = [&]() mutable - { - if (context) - return true; - - shared_context = Context::createShared(); - context = Context::createGlobal(shared_context.get()); - context->makeGlobalContext(); - context->setApplicationType(Context::ApplicationType::LOCAL); - - MainThreadStatus::getInstance(); - - registerAggregateFunctions(); - return true; - }; - - static bool initialized = initialize(); - (void) initialized; - total_memory_tracker.resetCounters(); total_memory_tracker.setHardLimit(1_GiB); CurrentThread::get().memory_tracker.resetCounters(); diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index a02ce66e6b5..c29efae1e7d 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -14,41 +14,37 @@ using namespace DB; + +ContextMutablePtr context; + +extern "C" int LLVMFuzzerInitialize(int *, char ***) +{ + if (context) + return true; + + SharedContextHolder shared_context = Context::createShared(); + context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + + registerInterpreters(); + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerDatabases(); + registerStorages(); + registerDictionaries(); + registerDisks(/* global_skip_access_check= */ true); + registerFormats(); + + return 0; +} + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try { std::string input = std::string(reinterpret_cast(data), size); - static SharedContextHolder shared_context; - static ContextMutablePtr context; - - auto initialize = [&]() mutable - { - if (context) - return true; - - shared_context = Context::createShared(); - context = Context::createGlobal(shared_context.get()); - context->makeGlobalContext(); - context->setApplicationType(Context::ApplicationType::LOCAL); - - registerInterpreters(); - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - registerDatabases(); - registerStorages(); - registerDictionaries(); - registerDisks(/* global_skip_access_check= */ true); - registerFormats(); - - return true; - }; - - static bool initialized = initialize(); - (void) initialized; - auto io = DB::executeQuery(input, context, QueryFlags{ .internal = true }, QueryProcessingStage::Complete).second; PullingPipelineExecutor executor(io.pipeline); From c4bdfd418d698bc9229fd7ce5ff34de96e6ffb6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 17:48:20 +0100 Subject: [PATCH 049/217] impl --- .../performance/views_max_insert_threads.xml | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/performance/views_max_insert_threads.xml b/tests/performance/views_max_insert_threads.xml index 2988984f5d8..473bcd02ab8 100644 --- a/tests/performance/views_max_insert_threads.xml +++ b/tests/performance/views_max_insert_threads.xml @@ -1,11 +1,11 @@ - - 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 756bde1158c4b3e6e65d324436291d53b9e25fbb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:27:15 +0100 Subject: [PATCH 050/217] 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 a70571762f7d73a7ecc94981e8086418ecfdeb3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 10:08:35 +0200 Subject: [PATCH 051/217] 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 3e6a1b99e023eb3d592c72c17ae4913a9074b5af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:05:45 +0200 Subject: [PATCH 052/217] 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 4d4fc8fd6f0123613305423d861429f54222d23f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:57:37 +0200 Subject: [PATCH 053/217] 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 054/217] `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 9a05a3ed9e7cccada42f49a0cd5c3896010f9edb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:34 +0200 Subject: [PATCH 055/217] 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 0f8feff4d3806fa6f81d24184ab68bcd6e727551 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:34:30 +0200 Subject: [PATCH 056/217] 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 057/217] 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 058/217] 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 d4537d91875d1ecec832af94fde15073c45a63d7 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:47 +0200 Subject: [PATCH 059/217] 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 060/217] 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 eb129b539fce2a407182d892ce3bd00f782a5833 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 13:46:27 +0200 Subject: [PATCH 061/217] 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 fd075470d6e141d5aa4d01ccefcc3d3ee04130c7 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 11:54:00 +0000 Subject: [PATCH 062/217] 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 063/217] 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 de99ee1b05e68b964535664d4197afd0944d0261 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 12:07:31 +0000 Subject: [PATCH 064/217] 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 8b52d7b711d54f1d4bb5b2f39bf4aea3966f64dc Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 13:35:19 +0100 Subject: [PATCH 065/217] 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 20faed85ca30c6352fd091e8d4d763fb98fe1311 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 15:18:37 +0200 Subject: [PATCH 066/217] 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 067/217] 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 068/217] 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 069/217] 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 070/217] 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 8d2b804c670d0941acc4fff059859017c1bd93c2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 21:57:08 +0100 Subject: [PATCH 071/217] 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 072/217] 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 bc312eb046db07d901e208cdc1bb0abb1df3eabd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 21:27:50 +0200 Subject: [PATCH 073/217] 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 074/217] 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 075/217] 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 076/217] 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 06863cf4157765c04759109afa756022dc5e9c55 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 31 Jul 2024 12:12:30 +0200 Subject: [PATCH 077/217] 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 078/217] 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 079/217] 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 9d14053cfe7867fd688c08b493c264ee679a4a61 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 Jul 2024 12:17:56 +0000 Subject: [PATCH 080/217] 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 081/217] 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 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 082/217] 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 083/217] 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 084/217] 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 31c142a96d49fbe1b46b21e4cdad366546dc7864 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:44:54 +0100 Subject: [PATCH 085/217] 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 7dbd3d75340522195e7d08a725cf5ae116288c8e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:51:38 +0000 Subject: [PATCH 086/217] 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 087/217] 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 9ffbd8f5073e180592a494742d1dc3af4427b55f Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:13:43 +0000 Subject: [PATCH 088/217] 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 1f1f0528ce3a1fb20ceee5513523787a14718b80 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 31 Jul 2024 14:32:07 +0000 Subject: [PATCH 089/217] 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 8c36fbf4eddeba9282b53f726976b55f62d3ee19 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:42:38 +0000 Subject: [PATCH 090/217] 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 091/217] 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 092/217] 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 093/217] 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 19cd00000373a5707178214744444b4d8c4034a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Jul 2024 17:18:55 +0200 Subject: [PATCH 094/217] 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 095/217] 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 096/217] 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 097/217] 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 098/217] 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 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 099/217] 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 100/217] 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 101/217] 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 102/217] 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 103/217] 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 cc27c254abd4b6fd8f64b47e0bdf6195041bd5ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 104/217] 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 105/217] 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 106/217] 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 107/217] 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 108/217] 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 f162d6bd5e03c6f717b4f45cf4c7ba6491aaa5fa Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Aug 2024 06:35:22 +0000 Subject: [PATCH 109/217] 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 110/217] 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 7db4065898633ace1f909711d4caeda8d135cace Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 10:30:50 +0200 Subject: [PATCH 111/217] 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 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 112/217] 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 113/217] 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 114/217] 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 115/217] 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 116/217] 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 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 117/217] 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 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 118/217] 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 119/217] 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 120/217] 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 121/217] 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 122/217] 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 123/217] 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 124/217] 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 125/217] 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 126/217] 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 127/217] 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 128/217] 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 129/217] 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 130/217] 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 131/217] 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 132/217] 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 133/217] 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 134/217] 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 135/217] 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 136/217] 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 137/217] 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 1e8d0d4a5e8d83a1d123a4b5b6c5a91b41caac1c Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 31 Jul 2024 06:09:14 +0000 Subject: [PATCH 138/217] 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 139/217] 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 30e0c1a1b8479e9b6be0701ba21e6050906a7e43 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 18:46:09 +0200 Subject: [PATCH 140/217] 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 141/217] 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 eac2c9fc3d8a88c1033e0f23e048421ecf4db850 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 142/217] 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 143/217] 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 144/217] 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 d6da86dad282e6ad176b115d4344944daa8b9756 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 26 Jun 2024 01:27:47 +0000 Subject: [PATCH 145/217] 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 146/217] 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 147/217] 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 148/217] 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 149/217] 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 150/217] 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 151/217] 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 152/217] 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 153/217] 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 154/217] 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 155/217] 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 156/217] 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 157/217] 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 158/217] 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 159/217] 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 160/217] 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 161/217] 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 162/217] 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 163/217] 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 164/217] 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 165/217] 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 166/217] 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 167/217] 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 168/217] 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 169/217] 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 170/217] empty From 6c8f458b0bf9981068c7fecfdd9cef627406419b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 11:13:41 +0200 Subject: [PATCH 171/217] 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 172/217] 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 173/217] 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 174/217] 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 02e48436057e45a884a1381e1c9cda9e1fe7de17 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 Jul 2024 09:30:39 +0000 Subject: [PATCH 175/217] 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 176/217] 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 177/217] 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 178/217] 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 179/217] 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 180/217] 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 a64e625f0ba20751f929222bd49aaf1295ff15ed Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 14:03:27 +0200 Subject: [PATCH 181/217] 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 f55784c636c1dcb503dadb2f75bd6b586271bf0d Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Aug 2024 12:07:05 +0000 Subject: [PATCH 182/217] 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 5256e8e6d08b5076be49b56fdebadb668892771c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 14:36:33 +0200 Subject: [PATCH 183/217] 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 83096249a7480b0bfa1d9246c17136727bba904c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 13:54:32 +0000 Subject: [PATCH 184/217] 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 aefed7cdd62e874f7507afe69d803c9164a283ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Aug 2024 16:06:53 +0200 Subject: [PATCH 185/217] 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 186/217] 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 187/217] 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 364622f567028ffc70785b681fc246d7151eef04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 188/217] 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 189/217] 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 190/217] 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 191/217] 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 b38c46a87d6eacbc7805562deb07ce586fd7e0fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 192/217] 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 193/217] 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 194/217] 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 195/217] 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 196/217] 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 197/217] 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 198/217] 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 199/217] 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 200/217] 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 201/217] 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 dd0ae04f90314ce6d5dbe748605e66f1a6d9024f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 202/217] 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 203/217] 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 204/217] 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 205/217] 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 206/217] 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 207/217] 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 208/217] 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 209/217] 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 210/217] 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 f97abf69949f8822d70f4b1251e1945f279dd0ec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 09:35:37 +0200 Subject: [PATCH 211/217] 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 212/217] 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 213/217] 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 fc651cc0c61feb37e9cf104612cc0ac0cd7448e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 17:57:48 +0200 Subject: [PATCH 214/217] 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 215/217] 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 216/217] 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 217/217] 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: