From 051290e6c912108986c896916db087c71230a121 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 24 Jun 2024 12:26:46 +0000 Subject: [PATCH 01/64] 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 02/64] 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 2e1f679ceb05afe4d5d813eb4048555c6311c3e1 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 16 Jul 2024 18:35:33 +0200 Subject: [PATCH 03/64] 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 2a893ed8cd10e6e3b7506f43b644b5037f96c49a Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 17:03:41 +0100 Subject: [PATCH 04/64] 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 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 05/64] 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 aaf603035e31874d6d5bcd024d0f4040715baa72 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 13:35:37 +0100 Subject: [PATCH 06/64] 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 13c93cc90cc35e33cad71095fa0300c5f4f1753e Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sun, 28 Jul 2024 10:29:52 +0000 Subject: [PATCH 07/64] 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 56135d18cb40506fc500b497068121bc96e3a9ee Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 11:23:01 +0000 Subject: [PATCH 08/64] 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 a70571762f7d73a7ecc94981e8086418ecfdeb3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 10:08:35 +0200 Subject: [PATCH 09/64] 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 10/64] 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 11/64] 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 2530c5eb41a759baded5380a4e697c2e884c0abd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:37:35 +0200 Subject: [PATCH 12/64] 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 13/64] 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 8b52d7b711d54f1d4bb5b2f39bf4aea3966f64dc Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 13:35:19 +0100 Subject: [PATCH 14/64] 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 15/64] 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 4aedb9d40298c1a3204bb72a3288ea711eb5e2f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 23:02:22 +0200 Subject: [PATCH 16/64] 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 17/64] 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 f9f17fb61e2ab27f90434b5e3fc9081c061eaae4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Jul 2024 12:32:17 +0200 Subject: [PATCH 18/64] 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 31c142a96d49fbe1b46b21e4cdad366546dc7864 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:44:54 +0100 Subject: [PATCH 19/64] 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 20/64] 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 9ffbd8f5073e180592a494742d1dc3af4427b55f Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:13:43 +0000 Subject: [PATCH 21/64] 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 8c36fbf4eddeba9282b53f726976b55f62d3ee19 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:42:38 +0000 Subject: [PATCH 22/64] 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 23/64] 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 24/64] 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 743d63767a74f41b3628c52ccf166be773baecf2 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 31 Jul 2024 17:06:49 +0000 Subject: [PATCH 25/64] 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 cc27c254abd4b6fd8f64b47e0bdf6195041bd5ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 26/64] 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 6403f3f545bee153ffaf4ce5bda6fcde33ef88d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 23:29:54 +0200 Subject: [PATCH 27/64] 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 1d85f9b1cba3c8fe168286a660d3c0a4fd471a95 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 1 Aug 2024 14:42:58 +0100 Subject: [PATCH 28/64] 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 0978441a0261c6003c7a9f4661ac87138e909622 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 29/64] 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 eac2c9fc3d8a88c1033e0f23e048421ecf4db850 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 30/64] 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 31/64] 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 9c05a0ad5a0269af02ae2234e1d01dc3ce64bce2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 09:34:32 +0100 Subject: [PATCH 32/64] 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 6c8f458b0bf9981068c7fecfdd9cef627406419b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 11:13:41 +0200 Subject: [PATCH 33/64] 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 62f0e09ecbb226ea72b5ee8d812436ef75038e33 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 12:17:08 +0200 Subject: [PATCH 34/64] 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: Fri, 2 Aug 2024 12:13:26 +0100 Subject: [PATCH 35/64] 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 7d45529fe8d28a6b39deb32d060343bb5d03b64f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 12:35:40 +0100 Subject: [PATCH 36/64] 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 37/64] 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 38/64] 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 aefed7cdd62e874f7507afe69d803c9164a283ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Aug 2024 16:06:53 +0200 Subject: [PATCH 39/64] 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 b3e2ce695514d4d314ed8ac1ecdb111c5f94ac7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 40/64] 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 41/64] 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 42/64] 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 b38c46a87d6eacbc7805562deb07ce586fd7e0fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 43/64] 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 44/64] 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 ce39957983af8bdd7d97e4a3729e2f97d3e0cb85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:11:25 +0200 Subject: [PATCH 45/64] 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 46/64] 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 47/64] 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 48/64] 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 a45ba44dbaa2ed43eb63e49fe609a01be978eac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 18:28:38 +0000 Subject: [PATCH 49/64] 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 50/64] 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 51/64] 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 52/64] 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 53/64] 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 54/64] 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 55/64] 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 56/64] 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 57/64] 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 58/64] 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 59/64] 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 60/64] 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 61/64] 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 62/64] 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 63/64] 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 64/64] 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;