From f0fdeb6d3ef9ae51098d11a0a22bb138b6a58df0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 24 Feb 2024 22:36:29 +0100 Subject: [PATCH 001/123] move adaptive task size calculation into base pool --- src/Core/Settings.h | 3 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 24 +-------- src/Storages/MergeTree/MergeTreeReadPool.cpp | 53 +++--------------- src/Storages/MergeTree/MergeTreeReadPool.h | 1 - .../MergeTree/MergeTreeReadPoolBase.cpp | 54 +++++++++++++++++-- .../MergeTree/MergeTreeReadPoolBase.h | 2 +- .../MergeTreeReadPoolParallelReplicas.cpp | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 1 + src/Storages/MergeTree/MergeTreeReadTask.h | 2 + 9 files changed, 64 insertions(+), 77 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d5ea9534e6c..a5fcb537539 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -765,7 +765,7 @@ class IColumn; M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ - M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \ + M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 2 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) ALIAS(filesystem_prefetch_min_bytes_for_single_read_task) \ M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \ M(Bool, merge_tree_determine_task_size_by_prewhere_columns, true, "Whether to use only prewhere columns size to determine reading task size.", 0) \ M(UInt64, merge_tree_compact_parts_min_granules_to_multibuffer_read, 16, "Only available in ClickHouse Cloud", 0) \ @@ -806,7 +806,6 @@ class IColumn; M(UInt64, prefetch_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the prefetch buffer to read from the filesystem.", 0) \ M(UInt64, filesystem_prefetch_step_bytes, 0, "Prefetch step in bytes. Zero means `auto` - approximately the best prefetch step will be auto deduced, but might not be 100% the best. The actual value might be different because of setting filesystem_prefetch_min_bytes_for_single_read_task", 0) \ M(UInt64, filesystem_prefetch_step_marks, 0, "Prefetch step in marks. Zero means `auto` - approximately the best prefetch step will be auto deduced, but might not be 100% the best. The actual value might be different because of setting filesystem_prefetch_min_bytes_for_single_read_task", 0) \ - M(UInt64, filesystem_prefetch_min_bytes_for_single_read_task, "2Mi", "Do not parallelize within one file read less than this amount of bytes. E.g. one reader will not receive a read task of size less than this amount. This setting is recommended to avoid spikes of time for aws getObject requests to aws", 0) \ M(UInt64, filesystem_prefetch_max_memory_usage, "1Gi", "Maximum memory usage for prefetches.", 0) \ M(UInt64, filesystem_prefetches_limit, 200, "Maximum number of prefetches. Zero means unlimited. A setting `filesystem_prefetches_max_memory_usage` is more recommended if you want to limit the number of prefetches", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index c19b4ddd8a2..b752c0d0d80 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -411,29 +411,7 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ 1, static_cast(std::round(static_cast(settings.filesystem_prefetch_step_bytes) / part_stat.approx_size_of_mark))); } - /// This limit is important to avoid spikes of slow aws getObject requests when parallelizing within one file. - /// (The default is taken from here https://docs.aws.amazon.com/whitepapers/latest/s3-optimizing-performance-best-practices/use-byte-range-fetches.html). - if (part_stat.approx_size_of_mark - && settings.filesystem_prefetch_min_bytes_for_single_read_task - && part_stat.approx_size_of_mark < settings.filesystem_prefetch_min_bytes_for_single_read_task) - { - const size_t min_prefetch_step_marks_by_total_cols = static_cast( - std::ceil(static_cast(settings.filesystem_prefetch_min_bytes_for_single_read_task) / part_stat.approx_size_of_mark)); - - /// At least one task to start working on it right now and another one to prefetch in the meantime. - const size_t new_min_prefetch_step_marks = std::min(min_prefetch_step_marks_by_total_cols, sum_marks / threads / 2); - if (min_prefetch_step_marks < new_min_prefetch_step_marks) - { - LOG_DEBUG(log, "Increasing min prefetch step from {} to {}", min_prefetch_step_marks, new_min_prefetch_step_marks); - min_prefetch_step_marks = new_min_prefetch_step_marks; - } - } - - if (part_stat.prefetch_step_marks < min_prefetch_step_marks) - { - LOG_DEBUG(log, "Increasing prefetch step from {} to {}", part_stat.prefetch_step_marks, min_prefetch_step_marks); - part_stat.prefetch_step_marks = min_prefetch_step_marks; - } + part_stat.prefetch_step_marks = std::max(part_stat.prefetch_step_marks, per_part_infos[i]->min_marks_per_task); LOG_DEBUG( log, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index e525f7f5f65..2f9ec317386 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -25,14 +25,6 @@ extern const int CANNOT_SCHEDULE_TASK; extern const int LOGICAL_ERROR; } -size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & columns_to_read) -{ - ColumnSize columns_size{}; - for (const auto & col_name : columns_to_read) - columns_size.add(part.getColumnSize(col_name)); - return columns_size.data_compressed; -} - MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, VirtualFields shared_virtual_fields_, @@ -53,38 +45,9 @@ MergeTreeReadPool::MergeTreeReadPool( column_names_, settings_, context_) - , min_marks_for_concurrent_read(pool_settings.min_marks_for_concurrent_read) , backoff_settings{context_->getSettingsRef()} , backoff_state{pool_settings.threads} { - if (std::ranges::count(is_part_on_remote_disk, true)) - { - const auto & settings = context_->getSettingsRef(); - - size_t total_compressed_bytes = 0; - size_t total_marks = 0; - for (const auto & part : parts_ranges) - { - const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info - ? prewhere_info->prewhere_actions->getRequiredColumnsNames() - : column_names_; - - total_compressed_bytes += getApproxSizeOfPart(*part.data_part, columns); - total_marks += part.getMarksCount(); - } - - if (total_marks) - { - const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; - const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); - /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - const auto heuristic_min_marks = std::min(total_marks / pool_settings.threads, min_bytes_per_task / avg_mark_bytes); - - if (heuristic_min_marks > min_marks_for_concurrent_read) - min_marks_for_concurrent_read = heuristic_min_marks; - } - } - fillPerThreadInfo(pool_settings.threads, pool_settings.sum_marks); } @@ -129,15 +92,16 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t task_idx, MergeTreeReadTa const auto part_idx = thread_task.part_idx; auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); + const auto min_marks_per_task = per_part_infos[part_idx]->min_marks_per_task; size_t need_marks; if (is_part_on_remote_disk[part_idx] && !pool_settings.use_const_size_tasks_for_remote_reading) need_marks = marks_in_part; else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_for_concurrent_read); + need_marks = std::min(marks_in_part, min_marks_per_task); /// Do not leave too little rows in part for next time. - if (marks_in_part > need_marks && marks_in_part - need_marks < min_marks_for_concurrent_read / 2) + if (marks_in_part > need_marks && marks_in_part - need_marks < min_marks_per_task / 2) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -256,8 +220,6 @@ void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks) parts_queue.push(std::move(info.second)); } - LOG_DEBUG(log, "min_marks_for_concurrent_read={}", min_marks_for_concurrent_read); - const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; for (size_t i = 0; i < threads && !parts_queue.empty(); ++i) @@ -270,15 +232,14 @@ void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks) auto & part_with_ranges = current_parts.back().part; size_t & marks_in_part = current_parts.back().sum_marks; const auto part_idx = current_parts.back().part_idx; + const auto min_marks_per_task = per_part_infos[part_idx]->min_marks_per_task; /// Do not get too few rows from part. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; + if (marks_in_part >= min_marks_per_task && need_marks < min_marks_per_task) + need_marks = min_marks_per_task; /// Do not leave too few rows in part for next time. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) + if (marks_in_part > need_marks && marks_in_part - need_marks < min_marks_per_task) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index cb0e8a9657f..b79af82ddb2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -78,7 +78,6 @@ private: void fillPerThreadInfo(size_t threads, size_t sum_marks); mutable std::mutex mutex; - size_t min_marks_for_concurrent_read = 0; /// State to track numbers of slow reads. struct BackoffState diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0cbb0a86b2f..397491a952d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -1,6 +1,7 @@ -#include -#include +#include #include +#include +#include namespace DB @@ -29,10 +30,53 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( , header(storage_snapshot->getSampleBlockForColumns(column_names)) , profile_callback([this](ReadBufferFromFileBase::ProfileInfo info_) { profileFeedback(info_); }) { - fillPerPartInfos(); + fillPerPartInfos(context_->getSettingsRef()); } -void MergeTreeReadPoolBase::fillPerPartInfos() +static size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & columns_to_read) +{ + ColumnSize columns_size{}; + for (const auto & col_name : columns_to_read) + columns_size.add(part.getColumnSize(col_name)); + return columns_size.data_compressed; +} + +static size_t calculateMinMarksPerTask( + const RangesInDataPart & part, + const Names & columns_to_read, + PrewhereInfoPtr prewhere_info, + const MergeTreeReadPoolBase::PoolSettings & pool_settings, + const Settings & settings) +{ + size_t min_marks_per_task = pool_settings.min_marks_for_concurrent_read; + const size_t part_marks_count = part.getMarksCount(); + if (part_marks_count && part.data_part->isStoredOnRemoteDisk()) + { + const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info + ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + : columns_to_read; + const size_t part_compressed_bytes = getApproxSizeOfPart(*part.data_part, columns); + + const auto avg_mark_bytes = std::max(part_compressed_bytes / part_marks_count, 1); + const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + const auto heuristic_min_marks + = std::min(pool_settings.sum_marks / pool_settings.threads / 2, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_per_task) + { + LOG_DEBUG( + &Poco::Logger::get("MergeTreeReadPoolBase"), + "Increasing min_marks_per_task from {} to {} based on columns size heuristic", + min_marks_per_task, + heuristic_min_marks); + min_marks_per_task = heuristic_min_marks; + } + } + LOG_DEBUG(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); + return min_marks_per_task; +} + +void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) { per_part_infos.reserve(parts_ranges.size()); is_part_on_remote_disk.reserve(parts_ranges.size()); @@ -83,6 +127,8 @@ void MergeTreeReadPoolBase::fillPerPartInfos() } is_part_on_remote_disk.push_back(part_with_ranges.data_part->isStoredOnRemoteDisk()); + read_task_info.min_marks_per_task + = calculateMinMarksPerTask(part_with_ranges, column_names, prewhere_info, pool_settings, settings); per_part_infos.push_back(std::make_shared(std::move(read_task_info))); } } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 1b5bfec5898..123f7538ba8 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -48,7 +48,7 @@ protected: const UncompressedCachePtr owned_uncompressed_cache; const Block header; - void fillPerPartInfos(); + void fillPerPartInfos(const Settings & settings); std::vector getPerPartSumMarks() const; MergeTreeReadTaskPtr createTask( diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 38035d97f56..d542abb2985 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -33,6 +33,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( context_) , extension(std::move(extension_)) , coordination_mode(CoordinationMode::Default) + , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) { extension.all_callback( InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 01c0a9f91be..e8228007f81 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -32,6 +32,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd context_) , extension(std::move(extension_)) , mode(mode_) + , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) { for (const auto & part : parts_ranges) request.push_back({part.data_part->info, MarkRanges{}}); diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index c8bb501c0e8..4c94a6afe64 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -66,6 +66,8 @@ struct MergeTreeReadTaskInfo MergeTreeBlockSizePredictorPtr shared_size_predictor; /// TODO: comment VirtualFields const_virtual_fields; + /// The amount of data to read per task based on size of the queried columns. + size_t min_marks_per_task = 0; }; using MergeTreeReadTaskInfoPtr = std::shared_ptr; From 85072356dbb4e298113d3a3a262374b1140fc8f5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 24 Feb 2024 22:42:21 +0100 Subject: [PATCH 002/123] support in parallel replicas --- .../MergeTree/MergeTreeReadPoolParallelReplicas.cpp | 9 ++++++--- .../MergeTree/MergeTreeReadPoolParallelReplicas.h | 1 + .../MergeTreeReadPoolParallelReplicasInOrder.cpp | 11 +++++------ .../MergeTreeReadPoolParallelReplicasInOrder.h | 1 + 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index d542abb2985..33eaf5a49bd 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -35,6 +35,9 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( , coordination_mode(CoordinationMode::Default) , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) { + for (const auto & info : per_part_infos) + min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + extension.all_callback( InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); } @@ -51,7 +54,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id auto result = extension.callback(ParallelReadRequest( coordination_mode, extension.number_of_current_replica, - pool_settings.min_marks_for_concurrent_read * pool_settings.threads, + min_marks_per_task * pool_settings.threads, /// For Default coordination mode we don't need to pass part names. RangesInDataPartsDescription{})); @@ -77,9 +80,9 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id MarkRanges ranges_to_read; size_t current_sum_marks = 0; - while (current_sum_marks < pool_settings.min_marks_for_concurrent_read && !current_task.ranges.empty()) + while (current_sum_marks < min_marks_per_task && !current_task.ranges.empty()) { - auto diff = pool_settings.min_marks_for_concurrent_read - current_sum_marks; + auto diff = min_marks_per_task - current_sum_marks; auto range = current_task.ranges.front(); if (range.getNumberOfMarks() > diff) { diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index ca159edb91c..6ba63cc2c9a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -32,6 +32,7 @@ private: const ParallelReadingExtension extension; const CoordinationMode coordination_mode; + size_t min_marks_per_task{0}; RangesInDataPartsDescription buffered_ranges; bool no_more_tasks_available{false}; LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index e8228007f81..6b5cf978423 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -34,6 +34,9 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd , mode(mode_) , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) { + for (const auto & info : per_part_infos) + min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + for (const auto & part : parts_ranges) request.push_back({part.data_part->info, MarkRanges{}}); @@ -77,12 +80,8 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t ta if (no_more_tasks) return nullptr; - auto response = extension.callback(ParallelReadRequest( - mode, - extension.number_of_current_replica, - pool_settings.min_marks_for_concurrent_read * request.size(), - request - )); + auto response + = extension.callback(ParallelReadRequest(mode, extension.number_of_current_replica, min_marks_per_task * request.size(), request)); if (!response || response->description.empty() || response->finish) { diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 4fe3f7a699c..22841bea212 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -30,6 +30,7 @@ private: const ParallelReadingExtension extension; const CoordinationMode mode; + size_t min_marks_per_task{0}; bool no_more_tasks{false}; RangesInDataPartsDescription request; RangesInDataPartsDescription buffered_tasks; From 281327b4ced9aadf421646095d71ad59ba6c70ba Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 26 Feb 2024 18:41:15 +0100 Subject: [PATCH 003/123] register settings change --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index f43ca154d56..6e38976b4b1 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, + {"24.4", {{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, + {"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, From e8e980932fdcc87934a3d4591d9a42130a4d60a4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 28 Feb 2024 20:06:12 +0100 Subject: [PATCH 004/123] fix --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bee42c3ddde..c4ba92d483e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -945,6 +945,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( PoolSettings pool_settings { + .threads = num_streams, + .sum_marks = parts_with_ranges.getMarksCountAllParts(), .min_marks_for_concurrent_read = info.min_marks_for_concurrent_read, .preferred_block_size_bytes = settings.preferred_block_size_bytes, .use_uncompressed_cache = info.use_uncompressed_cache, From 29729c0ea8fcf2543c8d1a2f996773b54829c86d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 23:42:37 +0100 Subject: [PATCH 005/123] use whole part size for Compact parts --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 397491a952d..2bf19e8cde5 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -38,7 +38,8 @@ static size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & ColumnSize columns_size{}; for (const auto & col_name : columns_to_read) columns_size.add(part.getColumnSize(col_name)); - return columns_size.data_compressed; + /// For compact parts we don't know individual column sizes, let's use whole part size as approximation + return columns_size.data_compressed ? columns_size.data_compressed : part.getBytesOnDisk(); } static size_t calculateMinMarksPerTask( From af6c4a3f2d247a3b39bd48f33e16758972e31723 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 22:09:17 +0000 Subject: [PATCH 006/123] fix noisy log --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 2bf19e8cde5..9d2484ecd04 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -65,7 +65,7 @@ static size_t calculateMinMarksPerTask( = std::min(pool_settings.sum_marks / pool_settings.threads / 2, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_per_task) { - LOG_DEBUG( + LOG_TEST( &Poco::Logger::get("MergeTreeReadPoolBase"), "Increasing min_marks_per_task from {} to {} based on columns size heuristic", min_marks_per_task, @@ -73,7 +73,7 @@ static size_t calculateMinMarksPerTask( min_marks_per_task = heuristic_min_marks; } } - LOG_DEBUG(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); + LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); return min_marks_per_task; } From bc1042a497257526c13a08d86fc736466cd454b5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 19:11:18 +0000 Subject: [PATCH 007/123] fix test --- tests/queries/0_stateless/02532_send_logs_level_test.reference | 1 + tests/queries/0_stateless/02532_send_logs_level_test.sh | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index 7e51b888d9c..72f4ea06184 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,3 +1,4 @@ + MergeTreeReadPoolBase: Will use min_marks_per_task=24 MergeTreeMarksLoader: Loading marks from path data.cmrk3 MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index 4afc6d4496b..08b3058c5a0 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -1,7 +1,8 @@ #!/usr/bin/env bash -# Tags: no-s3-storage, no-debug +# Tags: no-s3-storage, no-debug, no-random-merge-tree-settings # - no-s3-storage - S3 has additional logging # - no-debug - debug builds also has additional logging +# - no-random-merge-tree-settings - changes content of log messages CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 9b6bdee5f3f80661577e0204f622dd6e41571806 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 16 Jul 2024 13:44:50 +0100 Subject: [PATCH 008/123] one more change after merging with master --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8c096c13634..b78fed7e1c1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -66,6 +66,7 @@ static std::initializer_list Date: Tue, 16 Jul 2024 15:14:56 +0100 Subject: [PATCH 009/123] small fix --- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 94213e2ba61..26595fbb36d 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -379,7 +379,6 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ for (const auto & part : per_part_statistics) total_size_approx += part.sum_marks * part.approx_size_of_mark; - size_t min_prefetch_step_marks = pool_settings.min_marks_for_concurrent_read; for (size_t i = 0; i < per_part_infos.size(); ++i) { auto & part_stat = per_part_statistics[i]; @@ -411,11 +410,10 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ LOG_DEBUG( log, - "Sum marks: {}, threads: {}, min_marks_per_thread: {}, min prefetch step marks: {}, prefetches limit: {}, total_size_approx: {}", + "Sum marks: {}, threads: {}, min_marks_per_thread: {}, prefetches limit: {}, total_size_approx: {}", sum_marks, threads, min_marks_per_thread, - min_prefetch_step_marks, settings.filesystem_prefetches_limit, total_size_approx); From 288b0aaeb097c714e57d4cfd71a606b6942bd57c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 16 Jul 2024 15:16:48 +0100 Subject: [PATCH 010/123] fix build --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 79eaae14f59..0e713150625 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -1,7 +1,10 @@ -#include +#include + +#include #include #include -#include + +#include namespace DB From a0171256fd5cb6932ceb9b2cc98c9b553dfa82fd Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 19 Jul 2024 14:41:17 +0200 Subject: [PATCH 011/123] Small improvement for background pool in Keeper --- src/Coordination/KeeperServer.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index dc9658e895f..68515debe3b 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -368,7 +368,10 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co LockMemoryExceptionInThread::removeUniqueLock(); }; - asio_opts.thread_pool_size_ = getNumberOfPhysicalCPUCores(); + /// At least 16 threads for network communication in asio. + /// asio is async framework, so even with 1 thread it should be ok, but + /// still as safeguard it's better to have some redundant capacity here + asio_opts.thread_pool_size_ = std::max(16U, getNumberOfPhysicalCPUCores()); if (state_manager->isSecure()) { From 38126bb436c3f46cfdc321abca421f7fae969f5b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 19 Jul 2024 19:09:55 +0200 Subject: [PATCH 012/123] Add test. --- .../configs/disk_s3.xml | 22 +++++++++++++++ .../test_backup_restore_s3/test.py | 28 ++++++------------- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/configs/disk_s3.xml b/tests/integration/test_backup_restore_s3/configs/disk_s3.xml index 45a1e17b039..7ac1a052c30 100644 --- a/tests/integration/test_backup_restore_s3/configs/disk_s3.xml +++ b/tests/integration/test_backup_restore_s3/configs/disk_s3.xml @@ -21,6 +21,13 @@ minio123 33554432 + + s3_plain_rewritable + http://minio1:9001/root/data/disks/disk_s3_plain_rewritable/ + minio + minio123 + 33554432 + cache disk_s3 @@ -37,6 +44,20 @@ + + +
+ disk_s3_plain +
+
+
+ + +
+ disk_s3_plain_rewritable +
+
+
@@ -57,6 +78,7 @@ default disk_s3 disk_s3_plain + disk_s3_plain_rewritable disk_s3_cache disk_s3_other_bucket diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index d53335000a6..4840f5afc66 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -419,25 +419,15 @@ def test_backup_to_s3_multipart(): assert "ReadBufferFromS3RequestsErrors" not in restore_events -def test_backup_to_s3_native_copy(): - storage_policy = "policy_s3" - backup_name = new_backup_name() - backup_destination = ( - f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" - ) - (backup_events, restore_events) = check_backup_and_restore( - storage_policy, backup_destination - ) - # single part upload - assert backup_events["S3CopyObject"] > 0 - assert restore_events["S3CopyObject"] > 0 - assert node.contains_in_log( - f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" - ) - - -def test_backup_to_s3_native_copy_other_bucket(): - storage_policy = "policy_s3_other_bucket" +@pytest.mark.parametrize( + "storage_policy", + [ + "policy_s3", + "policy_s3_other_bucket", + "policy_s3_plain_rewritable", + ], +) +def test_backup_to_s3_native_copy(storage_policy): backup_name = new_backup_name() backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" From 5bea6751e07ef9753db0e532d476056679a4393a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 19 Jul 2024 19:11:07 +0200 Subject: [PATCH 013/123] Make the error message about broken parts more useful. --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 ++++++++++++++++- src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c2f87018872..3a44359b537 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -739,10 +739,25 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks } catch (...) { - /// Don't scare people with broken part error + /// Don't scare people with broken part error if it's retryable. if (!isRetryableException(std::current_exception())) + { LOG_ERROR(storage.log, "Part {} is broken and needs manual correction", getDataPartStorage().getFullPath()); + if (Exception * e = exception_cast(std::current_exception())) + { + /// Probably there is something wrong with files of this part. + /// So it can be helpful to add to the error message some information about those files. + String files_in_part; + for (auto it = getDataPartStorage().iterate(); it->isValid(); it->next()) + files_in_part += fmt::format("{}{} ({} bytes)", (files_in_part.empty() ? "" : ", "), it->name(), getDataPartStorage().getFileSize(it->name())); + if (!files_in_part.empty()) + e->addMessage("Part contains files: {}", files_in_part); + if (isEmpty()) + e->addMessage("Part is empty"); + } + } + // There could be conditions that data part to be loaded is broken, but some of meta infos are already written // into metadata before exception, need to clean them all. metadata_manager->deleteAll(/*include_projection*/ true); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 78a551591a6..12a4effe33c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5634,9 +5634,11 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r String part_name = part_info.getPartNameAndCheckFormat(format_version); auto backup = restored_parts_holder->getBackup(); + /// Find all files of this part in the backup. + Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true); + /// Calculate the total size of the part. UInt64 total_size_of_part = 0; - Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true); fs::path part_path_in_backup_fs = part_path_in_backup; for (const String & filename : filenames) total_size_of_part += backup->getFileSize(part_path_in_backup_fs / filename); From 7c89ca59af17576e3a4e6b42167c84b5045fb969 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Jul 2024 11:46:24 +0200 Subject: [PATCH 014/123] Bump From a3a4548d96dac4b480e3a54519cfa34fab17ce4c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 20 Jul 2024 15:01:29 +0200 Subject: [PATCH 015/123] Fix removing files after restoring to s3_plain_rewritable. --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index e7c85bea1c6..b5805f6d23a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -874,7 +874,9 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( /// Create metadata (see create_metadata_callback in DiskObjectStorageTransaction::writeFile()). if (mode == WriteMode::Rewrite) { - if (!object_storage.isWriteOnce() && metadata_storage.exists(path)) + /// Otherwise we will produce lost blobs which nobody points to + /// WriteOnce storages are not affected by the issue + if (!object_storage.isPlain() && metadata_storage.exists(path)) object_storage.removeObjectsIfExist(metadata_storage.getStorageObjects(path)); metadata_transaction->createMetadataFile(path, std::move(object_key), object_size); From 4e1afb9c4d30ffb217fe3ee1ced586eefeffaa76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 08:16:27 +0200 Subject: [PATCH 016/123] Fix bad test `01042_system_reload_dictionary_reloads_completely` --- ...ad_dictionary_reloads_completely.reference | 6 ++--- ...em_reload_dictionary_reloads_completely.sh | 22 ++++++++++++------- 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.reference b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.reference index f12dcd8258a..10bc7981d3e 100644 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.reference +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.reference @@ -1,6 +1,6 @@ 12 -> 102 13 -> 103 14 -> -1 -12(r) -> 102 -13(r) -> 103 -14(r) -> 104 +12 (after reloading) -> 102 +13 (after reloading) -> 103 +14 (after reloading) -> 104 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 2b075566ac3..42488ca946c 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 @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-random-settings +# Dictionaries are updated using the server time. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -6,8 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail -# NOTE: dictionaries TTLs works with server timezone, so session_timeout cannot be used -$CLICKHOUSE_CLIENT --session_timezone '' --multiquery < ', 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 +do + sleep 0.5 +done $CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" + +# By the way, the value for 14 is expected to not be updated at this moment, +# because the values were selected by the update field insert_time, and for 14 it was set as one day ago. $CLICKHOUSE_CLIENT --query "SELECT '14 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(14))" +# SYSTEM RELOAD DICTIONARY reloads it completely, regardless of the update field, so we will see new values, even for key 14. $CLICKHOUSE_CLIENT --query "SYSTEM RELOAD DICTIONARY '${CLICKHOUSE_DATABASE}.dict'" -$CLICKHOUSE_CLIENT --query "SELECT '12(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(12))" -$CLICKHOUSE_CLIENT --query "SELECT '13(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" -$CLICKHOUSE_CLIENT --query "SELECT '14(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(14))" +$CLICKHOUSE_CLIENT --query "SELECT '12 (after reloading) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(12))" +$CLICKHOUSE_CLIENT --query "SELECT '13 (after reloading) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" +$CLICKHOUSE_CLIENT --query "SELECT '14 (after reloading) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(14))" From de4a97a9a732945c3b3df4cc903a1dea30fffc5a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:41:15 +0200 Subject: [PATCH 017/123] Remove support for -WithDictionary suffix for data types --- src/DataTypes/DataTypeFactory.cpp | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index af37cde2846..eb3bc973857 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -125,23 +125,6 @@ DataTypePtr DataTypeFactory::getImpl(const String & family_name_param, const AST { String family_name = getAliasToOrName(family_name_param); - if (endsWith(family_name, "WithDictionary")) - { - ASTPtr low_cardinality_params = std::make_shared(); - String param_name = family_name.substr(0, family_name.size() - strlen("WithDictionary")); - if (parameters) - { - auto func = std::make_shared(); - func->name = param_name; - func->arguments = parameters; - low_cardinality_params->children.push_back(func); - } - else - low_cardinality_params->children.push_back(std::make_shared(param_name)); - - return getImpl("LowCardinality", low_cardinality_params); - } - const auto * creator = findCreatorByName(family_name); if constexpr (nullptr_on_error) { From f0f3bcfee972ba28928a9f33afd374d369d9babf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:45:57 +0200 Subject: [PATCH 018/123] Update tests --- ...cardinality_dictionary_deserialization.sql | 2 +- .../0_stateless/00688_low_cardinality_in.sql | 2 +- .../00688_low_cardinality_prewhere.sql | 2 +- .../00688_low_cardinality_serialization.sql | 4 +-- .../00688_low_cardinality_syntax.reference | 6 ----- .../00688_low_cardinality_syntax.sql | 26 +------------------ .../00717_low_cardinaliry_group_by.sql | 2 +- .../00718_low_cardinaliry_alter.sql | 2 +- .../00752_low_cardinality_mv_1.sql | 3 +-- .../02235_remote_fs_cache_stress.sh | 2 +- 10 files changed, 10 insertions(+), 41 deletions(-) diff --git a/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.sql b/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.sql index c4613acf5f3..d359efd8d42 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.sql +++ b/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.sql @@ -1,5 +1,5 @@ drop table if exists lc_dict_reading; -create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +create table lc_dict_reading (val UInt64, str LowCardinality(String), pat String) engine = MergeTree order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 1000000; select sum(toUInt64(str)), sum(toUInt64(pat)) from lc_dict_reading where val < 8129 or val > 8192 * 4; drop table if exists lc_dict_reading; diff --git a/tests/queries/0_stateless/00688_low_cardinality_in.sql b/tests/queries/0_stateless/00688_low_cardinality_in.sql index cb57fad51a4..c39fdb37160 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_in.sql +++ b/tests/queries/0_stateless/00688_low_cardinality_in.sql @@ -1,6 +1,6 @@ set allow_suspicious_low_cardinality_types = 1; drop table if exists lc_00688; -create table lc_00688 (str StringWithDictionary, val UInt8WithDictionary) engine = MergeTree order by tuple(); +create table lc_00688 (str LowCardinality(String), val LowCardinality(UInt8)) engine = MergeTree order by tuple(); insert into lc_00688 values ('a', 1), ('b', 2); select str, str in ('a', 'd') from lc_00688; select val, val in (1, 3) from lc_00688; diff --git a/tests/queries/0_stateless/00688_low_cardinality_prewhere.sql b/tests/queries/0_stateless/00688_low_cardinality_prewhere.sql index a15b2540fe6..17c74b7ca05 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_prewhere.sql +++ b/tests/queries/0_stateless/00688_low_cardinality_prewhere.sql @@ -1,5 +1,5 @@ drop table if exists lc_prewhere; -create table lc_prewhere (key UInt64, val UInt64, str StringWithDictionary, s String) engine = MergeTree order by key settings index_granularity = 8192; +create table lc_prewhere (key UInt64, val UInt64, str LowCardinality(String), s String) engine = MergeTree order by key settings index_granularity = 8192; insert into lc_prewhere select number, if(number < 10 or number > 8192 * 9, 1, 0), toString(number) as s, s from system.numbers limit 100000; select sum(toUInt64(str)), sum(toUInt64(s)) from lc_prewhere prewhere val == 1; drop table if exists lc_prewhere; diff --git a/tests/queries/0_stateless/00688_low_cardinality_serialization.sql b/tests/queries/0_stateless/00688_low_cardinality_serialization.sql index b4fe4b29200..1e4de3f3d3e 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_serialization.sql +++ b/tests/queries/0_stateless/00688_low_cardinality_serialization.sql @@ -8,8 +8,8 @@ select 'MergeTree'; drop table if exists lc_small_dict; drop table if exists lc_big_dict; -create table lc_small_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -create table lc_big_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +create table lc_small_dict (str LowCardinality(String)) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +create table lc_big_dict (str LowCardinality(String)) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_small_dict select toString(number % 1000) from system.numbers limit 1000000; insert into lc_big_dict select toString(number) from system.numbers limit 1000000; diff --git a/tests/queries/0_stateless/00688_low_cardinality_syntax.reference b/tests/queries/0_stateless/00688_low_cardinality_syntax.reference index ca27069a7df..b06beb0d6f0 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_syntax.reference +++ b/tests/queries/0_stateless/00688_low_cardinality_syntax.reference @@ -1,13 +1,7 @@ a a -a -a 1 1 -1 -1 -ab -ab ab ab - diff --git a/tests/queries/0_stateless/00688_low_cardinality_syntax.sql b/tests/queries/0_stateless/00688_low_cardinality_syntax.sql index a11d9e2d9fe..dccdac1d95d 100644 --- a/tests/queries/0_stateless/00688_low_cardinality_syntax.sql +++ b/tests/queries/0_stateless/00688_low_cardinality_syntax.sql @@ -13,56 +13,32 @@ drop table if exists lc_null_fix_str_0; drop table if exists lc_null_fix_str_1; create table lc_str_0 (str LowCardinality(String)) engine = Memory; -create table lc_str_1 (str StringWithDictionary) engine = Memory; create table lc_null_str_0 (str LowCardinality(Nullable(String))) engine = Memory; -create table lc_null_str_1 (str NullableWithDictionary(String)) engine = Memory; create table lc_int8_0 (val LowCardinality(Int8)) engine = Memory; -create table lc_int8_1 (val Int8WithDictionary) engine = Memory; create table lc_null_int8_0 (val LowCardinality(Nullable(Int8))) engine = Memory; -create table lc_null_int8_1 (val NullableWithDictionary(Int8)) engine = Memory; create table lc_fix_str_0 (str LowCardinality(FixedString(2))) engine = Memory; -create table lc_fix_str_1 (str FixedStringWithDictionary(2)) engine = Memory; create table lc_null_fix_str_0 (str LowCardinality(Nullable(FixedString(2)))) engine = Memory; -create table lc_null_fix_str_1 (str NullableWithDictionary(FixedString(2))) engine = Memory; insert into lc_str_0 select 'a'; -insert into lc_str_1 select 'a'; insert into lc_null_str_0 select 'a'; -insert into lc_null_str_1 select 'a'; insert into lc_int8_0 select 1; -insert into lc_int8_1 select 1; insert into lc_null_int8_0 select 1; -insert into lc_null_int8_1 select 1; insert into lc_fix_str_0 select 'ab'; -insert into lc_fix_str_1 select 'ab'; insert into lc_null_fix_str_0 select 'ab'; -insert into lc_null_fix_str_1 select 'ab'; select str from lc_str_0; -select str from lc_str_1; select str from lc_null_str_0; -select str from lc_null_str_1; select val from lc_int8_0; -select val from lc_int8_1; select val from lc_null_int8_0; -select val from lc_null_int8_1; select str from lc_fix_str_0; -select str from lc_fix_str_1; select str from lc_null_fix_str_0; -select str from lc_null_fix_str_1; drop table if exists lc_str_0; -drop table if exists lc_str_1; drop table if exists lc_null_str_0; -drop table if exists lc_null_str_1; drop table if exists lc_int8_0; -drop table if exists lc_int8_1; drop table if exists lc_null_int8_0; -drop table if exists lc_null_int8_1; drop table if exists lc_fix_str_0; -drop table if exists lc_fix_str_1; drop table if exists lc_null_fix_str_0; -drop table if exists lc_null_fix_str_1; select '-'; SELECT toLowCardinality('a') AS s, toTypeName(s), toTypeName(length(s)) from system.one; @@ -73,7 +49,7 @@ select (toLowCardinality(z) as val) || 'b' from (select arrayJoin(['c', 'd']) a select '-'; drop table if exists lc_str_uuid; -create table lc_str_uuid(str1 String, str2 LowCardinality(String), str3 StringWithDictionary) ENGINE=Memory; +create table lc_str_uuid(str1 String, str2 LowCardinality(String), str3 LowCardinality(String)) ENGINE=Memory; select toUUID(str1), toUUID(str2), toUUID(str3) from lc_str_uuid; select toUUID(str1, '', NULL), toUUID(str2, '', NULL), toUUID(str3, '', NULL) from lc_str_uuid; insert into lc_str_uuid values ('61f0c404-5cb3-11e7-907b-a6006ad3dba0', '61f0c404-5cb3-11e7-907b-a6006ad3dba0', '61f0c404-5cb3-11e7-907b-a6006ad3dba0'); diff --git a/tests/queries/0_stateless/00717_low_cardinaliry_group_by.sql b/tests/queries/0_stateless/00717_low_cardinaliry_group_by.sql index 02915d4e611..3115ab508fe 100644 --- a/tests/queries/0_stateless/00717_low_cardinaliry_group_by.sql +++ b/tests/queries/0_stateless/00717_low_cardinaliry_group_by.sql @@ -1,5 +1,5 @@ drop table if exists tab_00717; -create table tab_00717 (a String, b StringWithDictionary) engine = MergeTree order by a; +create table tab_00717 (a String, b LowCardinality(String)) engine = MergeTree order by a; insert into tab_00717 values ('a_1', 'b_1'), ('a_2', 'b_2'); select count() from tab_00717; select a from tab_00717 group by a order by a; diff --git a/tests/queries/0_stateless/00718_low_cardinaliry_alter.sql b/tests/queries/0_stateless/00718_low_cardinaliry_alter.sql index 591ff952132..524e396bcb1 100644 --- a/tests/queries/0_stateless/00718_low_cardinaliry_alter.sql +++ b/tests/queries/0_stateless/00718_low_cardinaliry_alter.sql @@ -7,7 +7,7 @@ alter table tab_00718 modify column b UInt32; select *, toTypeName(b) from tab_00718; alter table tab_00718 modify column b LowCardinality(UInt32); select *, toTypeName(b) from tab_00718; -alter table tab_00718 modify column b StringWithDictionary; +alter table tab_00718 modify column b LowCardinality(String); select *, toTypeName(b) from tab_00718; alter table tab_00718 modify column b LowCardinality(UInt32); select *, toTypeName(b) from tab_00718; diff --git a/tests/queries/0_stateless/00752_low_cardinality_mv_1.sql b/tests/queries/0_stateless/00752_low_cardinality_mv_1.sql index 60cc30ec2c3..4fdd0a2496e 100644 --- a/tests/queries/0_stateless/00752_low_cardinality_mv_1.sql +++ b/tests/queries/0_stateless/00752_low_cardinality_mv_1.sql @@ -1,7 +1,7 @@ drop table if exists lc_00752; drop table if exists lc_mv_00752; -create table lc_00752 (str StringWithDictionary) engine = MergeTree order by tuple(); +create table lc_00752 (str LowCardinality(String)) engine = MergeTree order by tuple(); insert into lc_00752 values ('a'), ('bbb'), ('ab'), ('accccc'), ('baasddas'), ('bcde'); @@ -12,4 +12,3 @@ select * from lc_mv_00752 order by letter; drop table if exists lc_00752; drop table if exists lc_mv_00752; - diff --git a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh index 0b6b9f461b0..ffc38c0c1bd 100755 --- a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh +++ b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh @@ -28,7 +28,7 @@ ORDER BY tuple(); INSERT INTO t_01411_num (num) SELECT number % 1000 FROM numbers(100000); -create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val; +create table lc_dict_reading (val UInt64, str LowCardinality(String), pat String) engine = MergeTree order by val; insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 100000; """ From 8217dcccc1438ec6186e8db53e17429a7060183f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 09:53:56 +0200 Subject: [PATCH 019/123] Stop ignoring SIGSEGV in GDB --- docker/test/fuzzer/run-fuzzer.sh | 1 - docker/test/stateless/attach_gdb.lib | 1 - 2 files changed, 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 6191aeaf304..b8f967ed9c2 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -208,7 +208,6 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass -handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index eb54f920b98..d288288bb17 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -20,7 +20,6 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass -handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue From 2f1818b8d5d49747faa8b496eb6c86d1d78b3ec7 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 22 Jul 2024 11:08:07 +0200 Subject: [PATCH 020/123] update error codes related to number of accepted arguments --- src/Functions/FunctionBase58Conversion.h | 3 +- src/Functions/FunctionChar.cpp | 3 +- src/Functions/hilbertEncode.cpp | 4 +-- src/Functions/mortonEncode.cpp | 4 +-- src/Functions/randDistribution.cpp | 3 +- src/Processors/Transforms/WindowTransform.cpp | 31 ++++++++++--------- .../test_functions.py | 13 ++++++-- .../0_stateless/02560_window_ntile.reference | 2 ++ .../0_stateless/02560_window_ntile.sql | 2 ++ .../0_stateless/03131_hilbert_coding.sql | 1 + 10 files changed, 42 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionBase58Conversion.h b/src/Functions/FunctionBase58Conversion.h index e519f9768cc..a34a36d8b81 100644 --- a/src/Functions/FunctionBase58Conversion.h +++ b/src/Functions/FunctionBase58Conversion.h @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } struct Base58Encode @@ -135,7 +136,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong number of arguments for function {}: 1 expected.", getName()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: 1 expected.", getName()); if (!isString(arguments[0].type)) throw Exception( diff --git a/src/Functions/FunctionChar.cpp b/src/Functions/FunctionChar.cpp index 0ebe1442f08..79e346a3ea4 100644 --- a/src/Functions/FunctionChar.cpp +++ b/src/Functions/FunctionChar.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } class FunctionChar : public IFunction @@ -36,7 +37,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} can't be {}, should be at least 1", getName(), arguments.size()); diff --git a/src/Functions/hilbertEncode.cpp b/src/Functions/hilbertEncode.cpp index 13512d0d36c..a4e3cc59b76 100644 --- a/src/Functions/hilbertEncode.cpp +++ b/src/Functions/hilbertEncode.cpp @@ -11,8 +11,8 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } @@ -87,7 +87,7 @@ public: return col_res; } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Illegal number of UInt arguments of function {}: should be not more than 2 dimensions", getName()); } diff --git a/src/Functions/mortonEncode.cpp b/src/Functions/mortonEncode.cpp index 0c19c7c3134..8cf0f18dbfe 100644 --- a/src/Functions/mortonEncode.cpp +++ b/src/Functions/mortonEncode.cpp @@ -16,8 +16,8 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } #define EXTRACT_VECTOR(INDEX) \ @@ -130,7 +130,7 @@ namespace ErrorCodes MASK(8, 6, col6->getUInt(i)), \ MASK(8, 7, col7->getUInt(i))) \ \ - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, \ + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, \ "Illegal number of UInt arguments of function {}, max: 8", \ getName()); \ diff --git a/src/Functions/randDistribution.cpp b/src/Functions/randDistribution.cpp index 4e616ada697..6a3dac748c1 100644 --- a/src/Functions/randDistribution.cpp +++ b/src/Functions/randDistribution.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -246,7 +247,7 @@ public: { auto desired = Distribution::getNumberOfArguments(); if (arguments.size() != desired && arguments.size() != desired + 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}. Should be {} or {}", getName(), desired, desired + 1); diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 86421adf4fb..2b255c5120e 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -56,7 +56,10 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } // Interface for true window functions. It's not much of an interface, they just @@ -1710,7 +1713,7 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); @@ -1723,7 +1726,7 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } @@ -1807,7 +1810,7 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); @@ -1820,7 +1823,7 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } @@ -1882,7 +1885,7 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); @@ -1895,7 +1898,7 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu { if (argument_types.size() != 1) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one argument", name_); } @@ -1968,7 +1971,7 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); @@ -1981,7 +1984,7 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } @@ -2116,7 +2119,7 @@ struct WindowFunctionNtile final : public StatefulWindowFunction : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) { if (argument_types.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} takes exactly one argument", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one argument", name_); auto type_id = argument_types[0]->getTypeId(); if (type_id != TypeIndex::UInt8 && type_id != TypeIndex::UInt16 && type_id != TypeIndex::UInt32 && type_id != TypeIndex::UInt64) @@ -2191,7 +2194,7 @@ namespace if (!buckets) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' funtcion must be greater than zero"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be greater than zero"); } } // new partition @@ -2404,7 +2407,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types.size() > 3) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function '{}' accepts at most 3 arguments, {} given", name, argument_types.size()); } @@ -2414,7 +2417,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { if (argument_types_.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} takes at least one argument", name_); } @@ -2504,7 +2507,7 @@ struct WindowFunctionNthValue final : public WindowFunction { if (argument_types_.size() != 2) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } @@ -2578,7 +2581,7 @@ struct NonNegativeDerivativeParams if (argument_types.size() != 2 && argument_types.size() != 3) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes 2 or 3 arguments", name_); } diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index fc03a77030e..3231fb87f33 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -75,11 +75,15 @@ def test_aggregate_states(start_cluster): except QueryRuntimeException as e: error_message = str(e) allowed_errors = [ - "NUMBER_OF_ARGUMENTS_DOESNT_MATCH", "ILLEGAL_TYPE_OF_ARGUMENT", # sequenceNextNode() and friends "UNKNOWN_AGGREGATE_FUNCTION", # Function X takes exactly one parameter: + "NUMBER_OF_ARGUMENTS_DOESNT_MATCH", + # Function X takes at least one argument + "TOO_FEW_ARGUMENTS_FOR_FUNCTION", + # Function X accepts at most 3 arguments, Y given + "TOO_MANY_ARGUMENTS_FOR_FUNCTION", # The function 'X' can only be used as a window function "BAD_ARGUMENTS", # aggThrow @@ -196,9 +200,7 @@ def test_string_functions(start_cluster): "Should start with ", # POINT/POLYGON/... "Cannot read input: expected a digit but got something else:", # ErrorCodes - "NUMBER_OF_ARGUMENTS_DOESNT_MATCH", "ILLEGAL_TYPE_OF_ARGUMENT", - "TOO_FEW_ARGUMENTS_FOR_FUNCTION", "DICTIONARIES_WAS_NOT_LOADED", "CANNOT_PARSE_UUID", "CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING", @@ -218,6 +220,11 @@ def test_string_functions(start_cluster): "CANNOT_PARSE_TEXT", "CANNOT_PARSE_DATETIME", # Function X takes exactly one parameter: + "NUMBER_OF_ARGUMENTS_DOESNT_MATCH", + # Function X takes at least one argument + "TOO_FEW_ARGUMENTS_FOR_FUNCTION", + # Function X accepts at most 3 arguments, Y given + "TOO_MANY_ARGUMENTS_FOR_FUNCTION", # The function 'X' can only be used as a window function "BAD_ARGUMENTS", # String foo is obviously not a valid IP address. diff --git a/tests/queries/0_stateless/02560_window_ntile.reference b/tests/queries/0_stateless/02560_window_ntile.reference index d877b2034cb..5f01832d075 100644 --- a/tests/queries/0_stateless/02560_window_ntile.reference +++ b/tests/queries/0_stateless/02560_window_ntile.reference @@ -213,6 +213,8 @@ select a, b, ntile('2') over (partition by a order by b) from(select intDiv(numb select a, b, ntile(0) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } select a, b, ntile(-2) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } select a, b, ntile(b + 1) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } +select a, b, ntile() over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select a, b, ntile(3, 2) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Bad window type select a, b, ntile(2) over (partition by a) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } select a, b, ntile(2) over (partition by a order by b rows between 4 preceding and unbounded following) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02560_window_ntile.sql b/tests/queries/0_stateless/02560_window_ntile.sql index 44e9b865052..d0e4d557e58 100644 --- a/tests/queries/0_stateless/02560_window_ntile.sql +++ b/tests/queries/0_stateless/02560_window_ntile.sql @@ -16,6 +16,8 @@ select a, b, ntile('2') over (partition by a order by b) from(select intDiv(numb select a, b, ntile(0) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } select a, b, ntile(-2) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } select a, b, ntile(b + 1) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } +select a, b, ntile() over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select a, b, ntile(3, 2) over (partition by a order by b) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Bad window type select a, b, ntile(2) over (partition by a) from(select intDiv(number,10) as a, number%10 as b from numbers(20)); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/03131_hilbert_coding.sql b/tests/queries/0_stateless/03131_hilbert_coding.sql index ed293dc6910..b16a0efad5d 100644 --- a/tests/queries/0_stateless/03131_hilbert_coding.sql +++ b/tests/queries/0_stateless/03131_hilbert_coding.sql @@ -46,6 +46,7 @@ drop table if exists hilbert_numbers_1_03131; select '----- ERRORS -----'; select hilbertEncode(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +select hilbertEncode(1, 2, 3); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION } select hilbertDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select hilbertEncode('text'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select hilbertDecode('text', 'text'); -- { serverError ILLEGAL_COLUMN } From 8246614f5e5cdec99941f048f51cb28b9eee03a5 Mon Sep 17 00:00:00 2001 From: zoomxi <419486879@qq.com> Date: Mon, 22 Jul 2024 17:57:30 +0800 Subject: [PATCH 021/123] throw if can't connect to any participating replicas --- src/Processors/QueryPlan/ReadFromRemote.cpp | 3 +- .../ParallelReplicasReadingCoordinator.cpp | 4 +-- .../ParallelReplicasReadingCoordinator.h | 4 +++ .../test.py | 35 +++++++++++++++++++ 4 files changed, 43 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index e27515a62a4..29e12c1e613 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -21,7 +21,7 @@ #include #include #include - +#include #include namespace DB @@ -429,6 +429,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder { shuffled_pool = shard.pool->getShuffledPools(current_settings); shuffled_pool.resize(max_replicas_to_use); + coordinator->adjustParticipatingReplicasCount(max_replicas_to_use); } else { diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f46b4de10b7..2ba66256116 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1031,7 +1031,7 @@ void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica if (!pimpl) { unavailable_nodes_registered_before_initialization.push_back(replica_number); - if (unavailable_nodes_registered_before_initialization.size() == replicas_count) + if (unavailable_nodes_registered_before_initialization.size() == participating_replicas_count) throw Exception(ErrorCodes::ALL_CONNECTION_TRIES_FAILED, "Can't connect to any replica chosen for query execution"); } else @@ -1061,7 +1061,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) } ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_) - : replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) + : replicas_count(replicas_count_), participating_replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) { } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 8b463fda395..c06ef6ef01a 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -30,11 +30,15 @@ public: /// needed to report total rows to read void setProgressCallback(ProgressCallback callback); + /// Participating replicas count may be less than replicas count in a shard. + void adjustParticipatingReplicasCount(size_t count) { participating_replicas_count = count; } + private: void initialize(CoordinationMode mode); std::mutex mutex; const size_t replicas_count{0}; + size_t participating_replicas_count{0}; size_t mark_segment_size{0}; std::unique_ptr pimpl; ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index 9f716459643..b77da338554 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -48,3 +48,38 @@ def test_skip_all_replicas(start_cluster, skip_unavailable_shards): "skip_unavailable_shards": skip_unavailable_shards, }, ) + +@pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) +def test_skip_all_participating_replicas1(start_cluster, skip_unavailable_shards): + cluster_name = "test_1_shard_3_unavaliable_replicas" + table_name = "tt1" + create_tables(cluster_name, table_name) + + with pytest.raises(QueryRuntimeException): + initiator.query( + f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 3, + "cluster_for_parallel_replicas": cluster_name, + "skip_unavailable_shards": skip_unavailable_shards, + "parallel_replicas_min_number_of_rows_per_replica": 500, + }, + ) + +@pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) +def test_skip_all_participating_replicas2(start_cluster, skip_unavailable_shards): + cluster_name = "test_1_shard_3_unavaliable_replicas" + table_name = "tt2" + create_tables(cluster_name, table_name) + + with pytest.raises(QueryRuntimeException): + initiator.query( + f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 2, + "cluster_for_parallel_replicas": cluster_name, + "skip_unavailable_shards": skip_unavailable_shards, + }, + ) From eb519c501622af6a6df6f3370b6209c92c2d4c20 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 20 Jul 2024 15:01:47 +0200 Subject: [PATCH 022/123] Change paths of temporary part directories during RESTORE use "/var/lib/clickhouse/data/test/table/tmp_restore_all_0_1_1_0-XXXXXXXX" instead of "/tmp/XXXXXXXX/data/test/table/0_1_1_0" (because directories can only be renamed in s3_plain_rewritable, and not moved to another parent directory). --- src/Disks/TemporaryFileOnDisk.cpp | 3 +- src/Disks/TemporaryFileOnDisk.h | 7 +++++ src/Storages/MergeTree/MergeTreeData.cpp | 36 ++++++++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 2 +- 4 files changed, 31 insertions(+), 17 deletions(-) diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 91eb214d941..88674e068d9 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -58,7 +58,8 @@ TemporaryFileOnDisk::~TemporaryFileOnDisk() if (!disk->exists(relative_path)) { - LOG_WARNING(getLogger("TemporaryFileOnDisk"), "Temporary path '{}' does not exist in '{}'", relative_path, disk->getPath()); + if (show_warning_if_removed) + LOG_WARNING(getLogger("TemporaryFileOnDisk"), "Temporary path '{}' does not exist in '{}'", relative_path, disk->getPath()); return; } diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index cccfc82cf9e..d0ff44c6f03 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -27,12 +27,19 @@ public: /// Return relative path (without disk) const String & getRelativePath() const { return relative_path; } + /// Sets whether the destructor should show a warning if the temporary file has been already removed. + /// By default a warning is shown. + void setShowWarningIfRemoved(bool show_warning_if_removed_) { show_warning_if_removed = show_warning_if_removed_; } + private: DiskPtr disk; /// Relative path in disk to the temporary file or directory String relative_path; + /// Whether the destructor should show a warning if the temporary file has been already removed. + bool show_warning_if_removed = true; + CurrentMetrics::Increment metric_increment; /// Specified if we know what for file is used (sort/aggregate/join). diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 12a4effe33c..642edf1cfb8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5549,12 +5549,17 @@ public: attachIfAllPartsRestored(); } - String getTemporaryDirectory(const DiskPtr & disk) + String getTemporaryDirectory(const DiskPtr & disk, const String & part_name) { std::lock_guard lock{mutex}; - auto it = temp_dirs.find(disk); - if (it == temp_dirs.end()) - it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/")).first; + auto it = temp_part_dirs.find(part_name); + if (it == temp_part_dirs.end()) + { + auto temp_part_dir = std::make_shared(disk, fs::path{storage->getRelativeDataPath()} / ("tmp_restore_" + part_name + "-")); + /// Attaching parts will rename them so it's expected for a temporary part directory not to exist anymore in the end. + temp_part_dir->setShowWarningIfRemoved(false); + it = temp_part_dirs.emplace(part_name, temp_part_dir).first; + } return it->second->getRelativePath(); } @@ -5572,7 +5577,7 @@ private: storage->attachRestoredParts(std::move(parts)); parts.clear(); - temp_dirs.clear(); + temp_part_dirs.clear(); num_parts = 0; } @@ -5581,7 +5586,7 @@ private: size_t num_parts = 0; size_t num_broken_parts = 0; MutableDataPartsVector parts; - std::map> temp_dirs; + std::map> temp_part_dirs; mutable std::mutex mutex; }; @@ -5648,11 +5653,9 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r /// Calculate paths, for example: /// part_name = 0_1_1_0 /// part_path_in_backup = /data/test/table/0_1_1_0 - /// tmp_dir = tmp/1aaaaaa - /// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0 + /// temp_part_dir = /var/lib/clickhouse/data/test/table/tmp_restore_all_0_1_1_0-XXXXXXXX auto disk = reservation->getDisk(); - fs::path temp_dir = restored_parts_holder->getTemporaryDirectory(disk); - fs::path temp_part_dir = temp_dir / part_path_in_backup_fs.relative_path(); + fs::path temp_part_dir = restored_parts_holder->getTemporaryDirectory(disk, part_name); /// Subdirectories in the part's directory. It's used to restore projections. std::unordered_set subdirs; @@ -5679,22 +5682,25 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r reservation->update(reservation->getSize() - file_size); } - if (auto part = loadPartRestoredFromBackup(disk, temp_part_dir.parent_path(), part_name, detach_if_broken)) + if (auto part = loadPartRestoredFromBackup(part_name, disk, temp_part_dir, detach_if_broken)) restored_parts_holder->addPart(part); else restored_parts_holder->increaseNumBrokenParts(); } -MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(const DiskPtr & disk, const String & temp_dir, const String & part_name, bool detach_if_broken) const +MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(const String & part_name, const DiskPtr & disk, const String & temp_part_dir, bool detach_if_broken) const { MutableDataPartPtr part; auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); + fs::path full_part_dir{temp_part_dir}; + String parent_part_dir = full_part_dir.parent_path(); + String part_dir_name = full_part_dir.filename(); - /// Load this part from the directory `tmp_part_dir`. + /// Load this part from the directory `temp_part_dir`. auto load_part = [&] { - MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, temp_dir, part_name); + MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, parent_part_dir, part_dir_name); builder.withPartFormatFromDisk(); part = std::move(builder).build(); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); @@ -5709,7 +5715,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons if (!part) { /// Make a fake data part only to copy its files to /detached/. - part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, temp_dir, part_name} + part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, parent_part_dir, part_dir_name} .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7076b680521..d185a5262fd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1473,7 +1473,7 @@ protected: /// Restores the parts of this table from backup. void restorePartsFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions); void restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup, bool detach_if_broken) const; - MutableDataPartPtr loadPartRestoredFromBackup(const DiskPtr & disk, const String & temp_dir, const String & part_name, bool detach_if_broken) const; + MutableDataPartPtr loadPartRestoredFromBackup(const String & part_name, const DiskPtr & disk, const String & temp_part_dir, bool detach_if_broken) const; /// Attaches restored parts to the storage. virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0; From 69be00cbf00b1be867760c996244093020da9034 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 15:47:53 +0000 Subject: [PATCH 023/123] Improve --- tests/integration/test_storage_s3/test.py | 78 +++++++++++------------ 1 file changed, 38 insertions(+), 40 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 40cbf4b44a6..65a5cdcad29 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -154,6 +154,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): def test_partition_by(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -161,26 +162,37 @@ def test_partition_by(started_cluster): values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_{_partition_id}.csv" put_query = f"""INSERT INTO TABLE FUNCTION - s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'CSV', '{table_format}') + s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{filename}', 'CSV', '{table_format}') PARTITION BY {partition_by} VALUES {values}""" run_query(instance, put_query) - assert "1,2,3\n" == get_s3_file_content(started_cluster, bucket, "test_3.csv") - assert "3,2,1\n" == get_s3_file_content(started_cluster, bucket, "test_1.csv") - assert "78,43,45\n" == get_s3_file_content(started_cluster, bucket, "test_45.csv") + assert "1,2,3\n" == get_s3_file_content(started_cluster, bucket, f"{id}/test_3.csv") + assert "3,2,1\n" == get_s3_file_content(started_cluster, bucket, f"{id}/test_1.csv") + assert "78,43,45\n" == get_s3_file_content( + started_cluster, bucket, f"{id}/test_45.csv" + ) filename = "test2_{_partition_id}.csv" instance.query( - f"create table p ({table_format}) engine=S3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'CSV') partition by column3" + f"create table p ({table_format}) engine=S3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{filename}', 'CSV') partition by column3" ) instance.query(f"insert into p values {values}") - assert "1,2,3\n" == get_s3_file_content(started_cluster, bucket, "test2_3.csv") - assert "3,2,1\n" == get_s3_file_content(started_cluster, bucket, "test2_1.csv") - assert "78,43,45\n" == get_s3_file_content(started_cluster, bucket, "test2_45.csv") + assert "1,2,3\n" == get_s3_file_content( + started_cluster, bucket, f"{id}/test2_3.csv" + ) + assert "3,2,1\n" == get_s3_file_content( + started_cluster, bucket, f"{id}/test2_1.csv" + ) + assert "78,43,45\n" == get_s3_file_content( + started_cluster, bucket, f"{id}/test2_45.csv" + ) + + instance.query("drop table p") def test_partition_by_string_column(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "col_num UInt32, col_str String" @@ -188,21 +200,20 @@ def test_partition_by_string_column(started_cluster): values = "(1, 'foo/bar'), (3, 'йцук'), (78, '你好')" filename = "test_{_partition_id}.csv" put_query = f"""INSERT INTO TABLE FUNCTION - s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'CSV', '{table_format}') + s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{filename}', 'CSV', '{table_format}') PARTITION BY {partition_by} VALUES {values}""" run_query(instance, put_query) assert '1,"foo/bar"\n' == get_s3_file_content( - started_cluster, bucket, "test_foo/bar.csv" - ) - assert '3,"йцук"\n' == get_s3_file_content(started_cluster, bucket, "test_йцук.csv") - assert '78,"你好"\n' == get_s3_file_content( - started_cluster, bucket, "test_你好.csv" + started_cluster, bucket, f"{id}/test_foo/bar.csv" ) + assert '3,"йцук"\n' == get_s3_file_content(started_cluster, bucket, f"{id}/test_йцук.csv") + assert '78,"你好"\n' == get_s3_file_content(started_cluster, bucket, f"{id}/test_你好.csv") def test_partition_by_const_column(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -211,12 +222,14 @@ def test_partition_by_const_column(started_cluster): values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test_{_partition_id}.csv" put_query = f"""INSERT INTO TABLE FUNCTION - s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'CSV', '{table_format}') + s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{filename}', 'CSV', '{table_format}') PARTITION BY {partition_by} VALUES {values}""" run_query(instance, put_query) - assert values_csv == get_s3_file_content(started_cluster, bucket, "test_88.csv") + assert values_csv == get_s3_file_content( + started_cluster, bucket, f"{id}/test_88.csv" + ) @pytest.mark.parametrize("special", ["space", "plus"]) @@ -276,46 +289,31 @@ def test_get_path_with_special(started_cluster, special): @pytest.mark.parametrize("auth", [pytest.param("'minio','minio123',", id="minio")]) def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster, str) -> None - + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" drop_empty_table_query = "DROP TABLE IF EXISTS empty_table" - create_empty_table_query = """ - CREATE TABLE empty_table ( - {} - ) ENGINE = Null() - """.format( - table_format + create_empty_table_query = ( + f"CREATE TABLE empty_table ({table_format}) ENGINE = Null()" ) run_query(instance, drop_empty_table_query) run_query(instance, create_empty_table_query) filename = "empty_put_test.csv" - put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') select * from empty_table".format( - started_cluster.minio_ip, - MINIO_INTERNAL_PORT, - bucket, - filename, - auth, - table_format, - ) + put_query = f"""insert into table function + s3('http://{started_cluster.minio_ip}:{MINIO_INTERNAL_PORT}/{bucket}/{id}/{filename}', {auth} 'CSV', '{table_format}') + select * from empty_table""" run_query(instance, put_query) assert ( run_query( instance, - "select count(*) from s3('http://{}:{}/{}/{}', {}'CSV', '{}')".format( - started_cluster.minio_ip, - MINIO_INTERNAL_PORT, - bucket, - filename, - auth, - table_format, - ), + f"""select count(*) from + s3('http://{started_cluster.minio_ip}:{MINIO_INTERNAL_PORT}/{bucket}/{id}/{filename}', {auth} 'CSV', '{table_format}')""", ) == "0\n" ) @@ -881,7 +879,7 @@ def test_storage_s3_get_unstable(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" - get_query = f"SELECT count(), sum(column3), sum(column4) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/test.csv', 'CSV', '{table_format}') FORMAT CSV" + get_query = f"SELECT count(), sum(column3), sum(column4) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/test.csv', 'CSV', '{table_format}') SETTINGS s3_max_single_read_retries=30 FORMAT CSV" result = run_query(instance, get_query) assert result.splitlines() == ["500001,500000,0"] From ac679892ab656788d9c9a23362ad82082e8620d4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:12:11 +0000 Subject: [PATCH 024/123] fix --- tests/integration/test_storage_s3/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 65a5cdcad29..84c887be388 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -497,6 +497,7 @@ def test_put_get_with_globs(started_cluster): def test_multipart(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster, str, bool) -> None + id = uuid.uuid4() bucket = ( started_cluster.minio_bucket if not maybe_auth @@ -519,7 +520,7 @@ def test_multipart(started_cluster, maybe_auth, positive): assert len(csv_data) > min_part_size_bytes - filename = "test_multipart.csv" + filename = f"{id}/test_multipart.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, @@ -691,7 +692,7 @@ def test_s3_glob_many_objects_under_selection(started_cluster): def create_files(thread_num): for f_num in range(thread_num * 63, thread_num * 63 + 63): path = f"folder1/file{f_num}.csv" - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, @@ -704,7 +705,7 @@ def test_s3_glob_many_objects_under_selection(started_cluster): jobs.append(threading.Thread(target=create_files, args=(thread_num,))) jobs[-1].start() - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, From fd2d56d58a6ebfd7c995bfc88fafde75dd12381f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jul 2024 19:05:49 +0200 Subject: [PATCH 025/123] Better processing of broken parts and their projections --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 ++++++-- .../MergeTree/MergeTreeSequentialSource.cpp | 3 ++- src/Storages/MergeTree/checkDataPart.cpp | 25 ++++++++++++------- 3 files changed, 26 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 8e73021d3e7..14c503d0420 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -224,14 +225,19 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write } catch (const Exception & e) { - if (e.code() != ErrorCodes::ABORTED && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM) + if (e.code() != ErrorCodes::ABORTED + && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM + && !isRetryableException(std::current_exception())) + { report_broken_part(); + } throw; } catch (...) { - report_broken_part(); + if (!isRetryableException(std::current_exception())) + report_broken_part(); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 07476e8b2e9..6eec03c4f18 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB { @@ -281,7 +282,7 @@ try catch (...) { /// Suspicion of the broken part. A part is added to the queue for verification. - if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + if (!isRetryableException(std::current_exception())) storage.reportBrokenPart(data_part); throw; } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 774fd95ebc6..922e6e8fb5d 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int CANNOT_ALLOCATE_MEMORY; extern const int CANNOT_MUNMAP; extern const int CANNOT_MREMAP; + extern const int CANNOT_SCHEDULE_TASK; extern const int UNEXPECTED_FILE_IN_DATA_PART; extern const int NO_FILE_IN_DATA_PART; extern const int NETWORK_ERROR; @@ -85,7 +86,8 @@ bool isRetryableException(std::exception_ptr exception_ptr) { return isNotEnoughMemoryErrorCode(e.code()) || e.code() == ErrorCodes::NETWORK_ERROR - || e.code() == ErrorCodes::SOCKET_TIMEOUT; + || e.code() == ErrorCodes::SOCKET_TIMEOUT + || e.code() == ErrorCodes::CANNOT_SCHEDULE_TASK; } catch (const Poco::Net::NetException &) { @@ -329,16 +331,21 @@ static IMergeTreeDataPart::Checksums checkDataPart( projections_on_disk.erase(projection_file); } - if (throw_on_broken_projection && !broken_projections_message.empty()) + if (throw_on_broken_projection) { - throw Exception(ErrorCodes::BROKEN_PROJECTION, "{}", broken_projections_message); - } + if (!broken_projections_message.empty()) + { + throw Exception(ErrorCodes::BROKEN_PROJECTION, "{}", broken_projections_message); + } - if (require_checksums && !projections_on_disk.empty()) - { - throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, - "Found unexpected projection directories: {}", - fmt::join(projections_on_disk, ",")); + /// This one is actually not broken, just redundant files on disk which + /// MergeTree will never use. + if (require_checksums && !projections_on_disk.empty()) + { + throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, + "Found unexpected projection directories: {}", + fmt::join(projections_on_disk, ",")); + } } if (is_cancelled()) From 454eb1af7ed69d00617e6bfa3daece508fcb565b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 17:15:05 +0000 Subject: [PATCH 026/123] Fix invalid format detection in schema inference tmp --- src/Storages/ObjectStorage/ReadBufferIterator.cpp | 4 +++- src/Storages/StorageFile.cpp | 8 ++++++-- src/Storages/StorageURL.cpp | 4 +++- .../03023_invalid_format_detection.reference | 1 + .../0_stateless/03023_invalid_format_detection.sh | 10 ++++++++++ 5 files changed, 23 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03023_invalid_format_detection.reference create mode 100755 tests/queries/0_stateless/03023_invalid_format_detection.sh diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 78cdc442f64..3d428073e88 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -162,7 +162,9 @@ ReadBufferIterator::Data ReadBufferIterator::next() { for (const auto & object_info : read_keys) { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName())) + auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName()); + /// Use this format only if we have a schema reader for it. + if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name)) { format = format_from_file_name; break; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..b4c03fa86c6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -416,7 +416,9 @@ namespace { for (const auto & path : paths) { - if (auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(path)) + auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(path); + /// Use this format only if we have a schema reader for it. + if (format_from_path && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_path)) { format = format_from_path; break; @@ -705,7 +707,9 @@ namespace /// If format is unknown we can try to determine it by the file name. if (!format) { - if (auto format_from_file = FormatFactory::instance().tryGetFormatFromFileName(*filename)) + auto format_from_file = FormatFactory::instance().tryGetFormatFromFileName(*filename); + /// Use this format only if we have a schema reader for it. + if (format_from_file && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file)) format = format_from_file; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 895da028fc2..4d82c2bfadb 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -735,7 +735,9 @@ namespace { for (const auto & url : options) { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(url)) + auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(url); + /// Use this format only if we have a schema reader for it. + if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name)) { format = format_from_file_name; break; diff --git a/tests/queries/0_stateless/03023_invalid_format_detection.reference b/tests/queries/0_stateless/03023_invalid_format_detection.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03023_invalid_format_detection.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03023_invalid_format_detection.sh b/tests/queries/0_stateless/03023_invalid_format_detection.sh new file mode 100755 index 00000000000..984a42aa218 --- /dev/null +++ b/tests/queries/0_stateless/03023_invalid_format_detection.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +touch $CLICKHOUSE_TEST_UNIQUE_NAME.xml +$CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.*')" 2>&1 | grep -c "CANNOT_DETECT_FORMAT" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.xml + From ef4cb8b28324f1599c082b70e9c9c516c6c519d4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 17:18:20 +0000 Subject: [PATCH 027/123] Add missing check --- src/Storages/ObjectStorage/ReadBufferIterator.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 3d428073e88..df78f128c80 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -222,7 +223,9 @@ ReadBufferIterator::Data ReadBufferIterator::next() { for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName())) + auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName()); + /// Use this format only if we have a schema reader for it. + if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name)) { format = format_from_file_name; break; From ea78ae85ed38e757f8dddd99d2018f01f3996f25 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jul 2024 19:23:45 +0200 Subject: [PATCH 028/123] Fix style --- src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 6eec03c4f18..311720728e7 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -20,12 +20,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int MEMORY_LIMIT_EXCEEDED; -} - - /// Lightweight (in terms of logic) stream for reading single part from /// MergeTree, used for merges and mutations. /// From 406addb2dcd0e82a71db06eb8ebaaa34bf1a543d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jul 2024 19:39:37 +0200 Subject: [PATCH 029/123] Add aborted to retryable errors --- src/Storages/MergeTree/DataPartsExchange.cpp | 3 +-- src/Storages/MergeTree/checkDataPart.cpp | 4 +++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 14c503d0420..061ee356203 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -225,8 +225,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write } catch (const Exception & e) { - if (e.code() != ErrorCodes::ABORTED - && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM + if (e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM && !isRetryableException(std::current_exception())) { report_broken_part(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 922e6e8fb5d..fb86d9e7603 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; extern const int BROKEN_PROJECTION; + extern const int ABORTED; } @@ -87,7 +88,8 @@ bool isRetryableException(std::exception_ptr exception_ptr) return isNotEnoughMemoryErrorCode(e.code()) || e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT - || e.code() == ErrorCodes::CANNOT_SCHEDULE_TASK; + || e.code() == ErrorCodes::CANNOT_SCHEDULE_TASK + || e.code() == ErrorCodes::ABORTED; } catch (const Poco::Net::NetException &) { From 07f642f1cac5253fda7eaa928ca5e659159b6b16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 03:13:52 +0200 Subject: [PATCH 030/123] Fix bad tests `share_big_sets`, CC @davenger --- ...en_multiple_mutations_tasks_long.reference | 1 + ..._between_multiple_mutations_tasks_long.sql | 13 +++++++---- ...sets_between_mutation_tasks_long.reference | 1 + ...e_big_sets_between_mutation_tasks_long.sql | 23 +++++++++++-------- 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference index 3a92fcf283d..9cb32105006 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference @@ -7,3 +7,4 @@ all_4_4_0 5000 all_2_2_0_9 5000 all_3_3_0_9 5000 all_4_4_0_9 +Ok diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index 741d0177971..631fd9cb2cc 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -18,12 +18,15 @@ SELECT name FROM system.parts WHERE database=currentDatabase() AND table = '0258 -- Start multiple mutations simultaneously SYSTEM STOP MERGES 02581_trips; -ALTER TABLE 02581_trips UPDATE description='5' WHERE id IN (SELECT (number*10 + 5)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=0; -ALTER TABLE 02581_trips UPDATE description='6' WHERE id IN (SELECT (number*10 + 6)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=0; -ALTER TABLE 02581_trips DELETE WHERE id IN (SELECT (number*10 + 7)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=0; -ALTER TABLE 02581_trips UPDATE description='8' WHERE id IN (SELECT (number*10 + 8)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=0; +ALTER TABLE 02581_trips UPDATE description='5' WHERE id IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; +ALTER TABLE 02581_trips UPDATE description='6' WHERE id IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; +ALTER TABLE 02581_trips DELETE WHERE id IN (SELECT (number*10 + 7)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; +ALTER TABLE 02581_trips UPDATE description='8' WHERE id IN (SELECT (number*10 + 8)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; SYSTEM START MERGES 02581_trips; -DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(200000000)); +DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(10000000)); SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part; +SYSTEM FLUSH LOGS; +SELECT DISTINCT peak_memory_usage < 2000000000 ? 'Ok' : toString(tuple(*)) FROM system.part_log WHERE database = currentDatabase() AND event_date >= yesterday() AND table = '02581_trips' AND event_type = 'MutatePart'; + DROP TABLE 02581_trips; diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference index 3a7410d925f..d21598bc12e 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference @@ -10,3 +10,4 @@ all_4_4_0 20000 16000 12000 +Ok diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql index b7314c8fa47..062f22357e8 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql @@ -18,42 +18,45 @@ SELECT count() from 02581_trips WHERE description = ''; SELECT name FROM system.parts WHERE database=currentDatabase() AND table = '02581_trips' AND active ORDER BY name; -- Run mutation with `id` a 'IN big subquery' -ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=2; +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; -ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10 + 1)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=2, max_rows_in_set=1000; +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10 + 1)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2, max_rows_in_set=1000; SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with func(`id`) IN big subquery -ALTER TABLE 02581_trips UPDATE description='b' WHERE id::UInt64 IN (SELECT (number*10 + 2)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=2; +ALTER TABLE 02581_trips UPDATE description='b' WHERE id::UInt64 IN (SELECT (number*10 + 2)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with non-PK `id2` IN big subquery -ALTER TABLE 02581_trips UPDATE description='c' WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(200000000)) SETTINGS mutations_sync=2; +ALTER TABLE 02581_trips UPDATE description='c' WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with PK and non-PK IN big subquery ALTER TABLE 02581_trips UPDATE description='c' WHERE - (id IN (SELECT (number*10 + 4)::UInt32 FROM numbers(200000000))) OR - (id2 IN (SELECT (number*10 + 4)::UInt32 FROM numbers(200000000))) + (id IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) OR + (id2 IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with PK and non-PK IN big subquery ALTER TABLE 02581_trips UPDATE description='c' WHERE - (id::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(200000000))) OR - (id2::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(200000000))) + (id::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) OR + (id2::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with PK and non-PK IN big subquery ALTER TABLE 02581_trips UPDATE description='c' WHERE - (id::UInt32 IN (SELECT (number*10 + 6)::UInt32 FROM numbers(200000000))) OR - ((id2+1)::String IN (SELECT (number*10 + 6)::UInt32 FROM numbers(200000000))) + (id::UInt32 IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) OR + ((id2+1)::String IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; +SYSTEM FLUSH LOGS; +SELECT DISTINCT peak_memory_usage < 2000000000 ? 'Ok' : toString(tuple(*)) FROM system.part_log WHERE database = currentDatabase() AND event_date >= yesterday() AND table = '02581_trips' AND event_type = 'MutatePart'; + DROP TABLE 02581_trips; From dee5790b22d06e3916f4030937eb9384247a6baa Mon Sep 17 00:00:00 2001 From: zoomxi <419486879@qq.com> Date: Tue, 23 Jul 2024 10:49:37 +0800 Subject: [PATCH 031/123] =?UTF-8?q?According=20to=20the=20suggestions=20fr?= =?UTF-8?q?om=20cr=EF=BC=8C=20modify=20test=20code?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../test.py | 26 +++---------------- 1 file changed, 4 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index b77da338554..e05f72316d0 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -49,8 +49,8 @@ def test_skip_all_replicas(start_cluster, skip_unavailable_shards): }, ) -@pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) -def test_skip_all_participating_replicas1(start_cluster, skip_unavailable_shards): +@pytest.mark.parametrize("max_parallel_replicas", [2, 3, 100]) +def test_skip_all_participating_replicas(start_cluster, max_parallel_replicas): cluster_name = "test_1_shard_3_unavaliable_replicas" table_name = "tt1" create_tables(cluster_name, table_name) @@ -60,26 +60,8 @@ def test_skip_all_participating_replicas1(start_cluster, skip_unavailable_shards f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", settings={ "allow_experimental_parallel_reading_from_replicas": 2, - "max_parallel_replicas": 3, + "max_parallel_replicas": max_parallel_replicas, "cluster_for_parallel_replicas": cluster_name, - "skip_unavailable_shards": skip_unavailable_shards, - "parallel_replicas_min_number_of_rows_per_replica": 500, - }, - ) - -@pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) -def test_skip_all_participating_replicas2(start_cluster, skip_unavailable_shards): - cluster_name = "test_1_shard_3_unavaliable_replicas" - table_name = "tt2" - create_tables(cluster_name, table_name) - - with pytest.raises(QueryRuntimeException): - initiator.query( - f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "max_parallel_replicas": 2, - "cluster_for_parallel_replicas": cluster_name, - "skip_unavailable_shards": skip_unavailable_shards, + "skip_unavailable_shards": 1, }, ) From feaa7ede979a624dfbe67facefe66395cfc021b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 06:34:29 +0200 Subject: [PATCH 032/123] Fix views over distributed tables with Analyzer --- src/Analyzer/IQueryTreeNode.h | 2 +- src/Storages/StorageDistributed.cpp | 37 ++++++++++------------------- 2 files changed, 14 insertions(+), 25 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index df3687f8fd9..b36c1401798 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -49,7 +49,7 @@ enum class QueryTreeNodeType : uint8_t /// Convert query tree node type to string const char * toString(QueryTreeNodeType type); -/** Query tree is semantical representation of query. +/** Query tree is a semantic representation of query. * Query tree node represent node in query tree. * IQueryTreeNode is base class for all query tree nodes. * diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 65323b4bb52..07892971ec2 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -43,7 +43,6 @@ #include #include -#include #include #include #include @@ -61,26 +60,20 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include #include #include #include #include -#include #include -#include #include -#include -#include #include #include @@ -90,7 +83,6 @@ #include #include #include -#include #include #include #include @@ -496,7 +488,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( } std::optional optimized_stage; - if (settings.allow_experimental_analyzer) + if (query_info.query_tree) optimized_stage = getOptimizedQueryProcessingStageAnalyzer(query_info, settings); else optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); @@ -860,31 +852,28 @@ void StorageDistributed::read( modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); + + /// Return directly (with correct header) if no shard to query. + if (modified_query_info.getCluster()->getShardsInfo().empty()) + return; } else { header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - if (!settings.allow_experimental_analyzer) - { modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr); - } - /// Return directly (with correct header) if no shard to query. - if (modified_query_info.getCluster()->getShardsInfo().empty()) - { - if (settings.allow_experimental_analyzer) + if (modified_query_info.getCluster()->getShardsInfo().empty()) + { + Pipe pipe(std::make_shared(header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); + query_plan.addStep(std::move(read_from_pipe)); + return; - - Pipe pipe(std::make_shared(header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); - query_plan.addStep(std::move(read_from_pipe)); - - return; + } } const auto & snapshot_data = assert_cast(*storage_snapshot->data); From ac13983ebb8f2a75ed44f1e2616e1aac006ce69d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 06:52:19 +0200 Subject: [PATCH 033/123] Better --- .../IInterpreterUnionOrSelectQuery.cpp | 18 +++++++++++------ src/Interpreters/InterpreterSelectQuery.cpp | 20 ++++++++----------- src/Interpreters/InterpreterSelectQuery.h | 1 - src/Storages/TTLDescription.cpp | 2 +- 4 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index d8f6df05ca4..bc7b07ca18d 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -17,13 +17,19 @@ namespace DB { -IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const DB::ASTPtr& query_ptr_, - const DB::ContextMutablePtr& context_, const DB::SelectQueryOptions& options_) - : query_ptr(query_ptr_) - , context(context_) - , options(options_) - , max_streams(context->getSettingsRef().max_threads) +IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, + const ContextMutablePtr & context_, const SelectQueryOptions & options_) + : query_ptr(query_ptr_) + , context(context_) + , options(options_) + , max_streams(context->getSettingsRef().max_threads) { + /// FIXME All code here will work with the old analyzer, however for views over Distributed tables + /// it's possible that new analyzer will be enabled in ::getQueryProcessingStage method + /// of the underlying storage when all other parts of infrastructure are not ready for it + /// (built with old analyzer). + context->setSetting("allow_experimental_analyzer", false); + if (options.shard_num) context->addSpecialScalar( "_shard_num", diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7bee497f6da..0f24888cb79 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -75,7 +75,6 @@ #include #include -#include #include #include #include @@ -214,11 +213,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( {} InterpreterSelectQuery::InterpreterSelectQuery( - const ASTPtr & query_ptr_, - const ContextPtr & context_, - Pipe input_pipe_, - const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) + const ASTPtr & query_ptr_, + const ContextPtr & context_, + Pipe input_pipe_, + const SelectQueryOptions & options_) + : InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) {} InterpreterSelectQuery::InterpreterSelectQuery( @@ -227,18 +226,15 @@ InterpreterSelectQuery::InterpreterSelectQuery( const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, const SelectQueryOptions & options_) - : InterpreterSelectQuery( - query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) -{ -} + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) +{} InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const ContextPtr & context_, const SelectQueryOptions & options_, PreparedSetsPtr prepared_sets_) - : InterpreterSelectQuery( - query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, prepared_sets_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, prepared_sets_) {} InterpreterSelectQuery::~InterpreterSelectQuery() = default; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index d4ed19d45ea..ac1230a6eba 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -26,7 +26,6 @@ class Logger; namespace DB { -class SubqueryForSet; class InterpreterSelectWithUnionQuery; class Context; class QueryPlan; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index ac091e7cf3c..92f6f17583d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -172,7 +172,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType /// with subqueries it's possible that new analyzer will be enabled in ::read method /// of underlying storage when all other parts of infra are not ready for it /// (built with old analyzer). - context_copy->setSetting("allow_experimental_analyzer", Field{0}); + context_copy->setSetting("allow_experimental_analyzer", false); auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); From d50742eadc1234bf75fa76e7b21aec0f8763851e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 06:52:34 +0200 Subject: [PATCH 034/123] Better --- src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index bc7b07ca18d..f64b9540dbb 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -24,7 +24,7 @@ IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const ASTPtr & qu , options(options_) , max_streams(context->getSettingsRef().max_threads) { - /// FIXME All code here will work with the old analyzer, however for views over Distributed tables + /// FIXME All code here will work with the old analyzer, however for views over Distributed tables /// it's possible that new analyzer will be enabled in ::getQueryProcessingStage method /// of the underlying storage when all other parts of infrastructure are not ready for it /// (built with old analyzer). From 1c723e7fd7f85cb8bdd4b6103a158229912af164 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Jul 2024 09:10:47 +0000 Subject: [PATCH 035/123] black --- tests/integration/test_storage_s3/test.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 84c887be388..5453ad5a796 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -208,8 +208,12 @@ def test_partition_by_string_column(started_cluster): assert '1,"foo/bar"\n' == get_s3_file_content( started_cluster, bucket, f"{id}/test_foo/bar.csv" ) - assert '3,"йцук"\n' == get_s3_file_content(started_cluster, bucket, f"{id}/test_йцук.csv") - assert '78,"你好"\n' == get_s3_file_content(started_cluster, bucket, f"{id}/test_你好.csv") + assert '3,"йцук"\n' == get_s3_file_content( + started_cluster, bucket, f"{id}/test_йцук.csv" + ) + assert '78,"你好"\n' == get_s3_file_content( + started_cluster, bucket, f"{id}/test_你好.csv" + ) def test_partition_by_const_column(started_cluster): @@ -895,9 +899,10 @@ def test_storage_s3_get_slow(started_cluster): def test_storage_s3_put_uncompressed(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] - filename = "test_put_uncompressed.bin" + filename = f"{id}/test_put_uncompressed.bin" name = "test_put_uncompressed" data = [ "'Gloria Thompson',99", @@ -949,6 +954,7 @@ def test_storage_s3_put_uncompressed(started_cluster): r = result.strip().split("\t") assert int(r[0]) >= 1, blob_storage_log assert all(col == r[0] for col in r), blob_storage_log + run_query(instance, f"DROP TABLE {name}") @pytest.mark.parametrize( From a000f8f8a13598d8c4dd24043d5029d7a0158ace Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 23 Jul 2024 11:27:16 +0200 Subject: [PATCH 036/123] fix stateless test 02346_non_negative_derivative --- tests/queries/0_stateless/02346_non_negative_derivative.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.sql b/tests/queries/0_stateless/02346_non_negative_derivative.sql index 704241da16c..ab648f2ee73 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.sql +++ b/tests/queries/0_stateless/02346_non_negative_derivative.sql @@ -58,8 +58,8 @@ SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 13 YEAR) OVER (PAR -- test against wrong arguments/types SELECT ts, metric, nonNegativeDerivative(metric, 1, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } SELECT ts, metric, nonNegativeDerivative('string not datetime', ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } -SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- cleanup DROP TABLE IF EXISTS nnd; From 7750914f0b85e324dbf5d49d565312f117c02a2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 23 Jul 2024 09:33:58 +0000 Subject: [PATCH 037/123] Decrease rate limit In slow tests (sanitizer, debug) it can take 7-8 seconds to start up `clickhouse-client`. In those cases the rate limit doesn't make the wait time longer, because the input will be already in the input of clickhouse-client when it starts to execute the query. --- .../0_stateless/01923_network_receive_time_metric_insert.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index 4d7e79fae52..a6b83eba27d 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" # Rate limit is chosen for operation to spent more than one second. -seq 1 1000 | pv --quiet --rate-limit 500 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" +seq 1 1000 | pv --quiet --rate-limit 400 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" # We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client. ${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; From 342efff2edd9bcbecd28f5196fb11dcabdb629e1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Jul 2024 12:49:50 +0200 Subject: [PATCH 038/123] Bump From ba3979275b53a44d9a9c097f047188b2d44ffa8b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 23 Jul 2024 13:54:18 +0200 Subject: [PATCH 039/123] Test that big set is shared between parts by checking read_rows from system.part_log --- ...en_multiple_mutations_tasks_long.reference | 5 +++- ..._between_multiple_mutations_tasks_long.sql | 26 ++++++++++++++++--- ...sets_between_mutation_tasks_long.reference | 9 ++++++- ...e_big_sets_between_mutation_tasks_long.sql | 19 +++++++++++++- 4 files changed, 53 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference index 9cb32105006..d3e7e113ffe 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.reference @@ -3,8 +3,11 @@ all_1_1_0 all_2_2_0 all_3_3_0 all_4_4_0 +40000 5000 all_1_1_0_9 5000 all_2_2_0_9 5000 all_3_3_0_9 5000 all_4_4_0_9 -Ok +mutation_version has_parts_for_which_set_was_built has_parts_that_shared_set +8 1 1 +9 1 1 diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index 631fd9cb2cc..40f7800fee1 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -23,10 +23,30 @@ ALTER TABLE 02581_trips UPDATE description='6' WHERE id IN (SELECT (number*10 + ALTER TABLE 02581_trips DELETE WHERE id IN (SELECT (number*10 + 7)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; ALTER TABLE 02581_trips UPDATE description='8' WHERE id IN (SELECT (number*10 + 8)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=0; SYSTEM START MERGES 02581_trips; -DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(10000000)); -SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part; + +-- Wait for mutations to finish +SELECT count() FROM 02581_trips SETTINGS select_sequential_consistency = 1; + +DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(10000000)) SETTINGS lightweight_deletes_sync = 2; +SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part SETTINGS select_sequential_consistency=1; SYSTEM FLUSH LOGS; -SELECT DISTINCT peak_memory_usage < 2000000000 ? 'Ok' : toString(tuple(*)) FROM system.part_log WHERE database = currentDatabase() AND event_date >= yesterday() AND table = '02581_trips' AND event_type = 'MutatePart'; + +-- Check that in every mutation there were parts where selected rows count then the size of big sets which will mean that sets were shared +-- Also check that there was at least one part that read more rows then the size of set which will mean that the +WITH 10000000 AS rows_in_set +SELECT + mutation_version, + countIf(read_rows >= rows_in_set) >= 1 as has_parts_for_which_set_was_built, + countIf(read_rows <= rows_in_set) >= 1 as has_parts_that_shared_set +FROM +( + SELECT + CAST(splitByChar('_', part_name)[5], 'UInt64') AS mutation_version, + read_rows + FROM system.part_log + WHERE database = currentDatabase() and (event_date >= yesterday()) AND (`table` = '02581_trips') AND (event_type = 'MutatePart') +) +GROUP BY mutation_version ORDER BY mutation_version FORMAT TSVWithNames; DROP TABLE 02581_trips; diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference index d21598bc12e..eecd768eb5e 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.reference @@ -10,4 +10,11 @@ all_4_4_0 20000 16000 12000 -Ok +mutation_version has_parts_for_which_set_was_built has_parts_that_shared_set +5 1 1 +6 1 1 +7 1 1 +8 1 1 +9 1 1 +10 1 1 +11 1 1 diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql index 062f22357e8..603c7cb7db0 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql @@ -29,6 +29,8 @@ ALTER TABLE 02581_trips UPDATE description='b' WHERE id::UInt64 IN (SELECT (numb SELECT count() from 02581_trips WHERE description = ''; -- Run mutation with non-PK `id2` IN big subquery +--SELECT count(), _part FROM 02581_trips WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)) GROUP BY _part ORDER BY _part; +--EXPLAIN SELECT (), _part FROM 02581_trips WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)); ALTER TABLE 02581_trips UPDATE description='c' WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; @@ -57,6 +59,21 @@ SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; SYSTEM FLUSH LOGS; -SELECT DISTINCT peak_memory_usage < 2000000000 ? 'Ok' : toString(tuple(*)) FROM system.part_log WHERE database = currentDatabase() AND event_date >= yesterday() AND table = '02581_trips' AND event_type = 'MutatePart'; +-- Check that in every mutation there were parts where selected rows count then the size of big sets which will mean that sets were shared +-- Also check that there was at least one part that read more rows then the size of set which will mean that the +WITH 10000000 AS rows_in_set +SELECT + mutation_version, + countIf(read_rows >= rows_in_set) >= 1 as has_parts_for_which_set_was_built, + countIf(read_rows <= rows_in_set) >= 1 as has_parts_that_shared_set +FROM +( + SELECT + CAST(splitByChar('_', part_name)[5], 'UInt64') AS mutation_version, + read_rows + FROM system.part_log + WHERE database = currentDatabase() and (event_date >= yesterday()) AND (`table` = '02581_trips') AND (event_type = 'MutatePart') +) +GROUP BY mutation_version ORDER BY mutation_version FORMAT TSVWithNames; DROP TABLE 02581_trips; From ea61af961acd3d8fb9dd3ee2b48900e4f1ee7937 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 00:14:19 +0100 Subject: [PATCH 040/123] impl --- src/Server/TCPHandler.cpp | 73 +++++++++++++++++++++++---------------- 1 file changed, 43 insertions(+), 30 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 07366d7cc07..2a2e2e22538 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,48 +1,48 @@ -#include -#include -#include #include #include #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include +#include +#include +#include #include #include -#include -#include +#include +#include +#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include #include -#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include #include -#include #include #include @@ -61,6 +61,8 @@ #include +#include + using namespace std::literals; using namespace DB; @@ -1036,6 +1038,17 @@ void TCPHandler::processOrdinaryQuery() PullingAsyncPipelineExecutor executor(pipeline); CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread}; + /// The following may happen: + /// * current thread is holding the lock + /// * because of the exception we unwind the stack and call the destructor of `executor` + /// * the destructor calls cancel() and waits for all query threads to finish + /// * at the same time one of the query threads is trying to acquire the lock, e.g. inside `merge_tree_read_task_callback` + /// * deadlock + SCOPE_EXIT({ + if (out_lock.owns_lock()) + out_lock.unlock(); + }); + Block block; while (executor.pull(block, interactive_delay / 1000)) { @@ -1079,8 +1092,7 @@ void TCPHandler::processOrdinaryQuery() } /// This lock wasn't acquired before and we make .lock() call here - /// so everything under this line is covered even together - /// with sendProgress() out of the scope + /// so everything under this line is covered. out_lock.lock(); /** If data has run out, we will send the profiling data and total values to @@ -1107,6 +1119,7 @@ void TCPHandler::processOrdinaryQuery() last_sent_snapshots.clear(); } + out_lock.lock(); sendProgress(); } From 496c1fbf8413db3b3a3b410750adf3b5ecddabf3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jul 2024 12:46:09 +0000 Subject: [PATCH 041/123] Un-flake test_runtime_configurable_cache_size --- ..._query_cache.xml => empty_query_cache.xml} | 2 +- .../test.py | 60 +++++++++---------- ...query_cache_asynchronous_metrics.reference | 2 + ...02494_query_cache_asynchronous_metrics.sql | 13 ++++ 4 files changed, 45 insertions(+), 32 deletions(-) rename tests/integration/test_runtime_configurable_cache_size/configs/{smaller_query_cache.xml => empty_query_cache.xml} (64%) create mode 100644 tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.sql diff --git a/tests/integration/test_runtime_configurable_cache_size/configs/smaller_query_cache.xml b/tests/integration/test_runtime_configurable_cache_size/configs/empty_query_cache.xml similarity index 64% rename from tests/integration/test_runtime_configurable_cache_size/configs/smaller_query_cache.xml rename to tests/integration/test_runtime_configurable_cache_size/configs/empty_query_cache.xml index 6f2de0fa8f5..c4872a0ce41 100644 --- a/tests/integration/test_runtime_configurable_cache_size/configs/smaller_query_cache.xml +++ b/tests/integration/test_runtime_configurable_cache_size/configs/empty_query_cache.xml @@ -1,7 +1,7 @@ - 1 + 0 diff --git a/tests/integration/test_runtime_configurable_cache_size/test.py b/tests/integration/test_runtime_configurable_cache_size/test.py index f761005f297..26676009e19 100644 --- a/tests/integration/test_runtime_configurable_cache_size/test.py +++ b/tests/integration/test_runtime_configurable_cache_size/test.py @@ -94,51 +94,49 @@ CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs") def test_query_cache_size_is_runtime_configurable(start_cluster): - # the initial config specifies the maximum query cache size as 2, run 3 queries, expect 2 cache entries node.query("SYSTEM DROP QUERY CACHE") + + # the initial config allows at most two query cache entries but we don't mind node.query("SELECT 1 SETTINGS use_query_cache = 1, query_cache_ttl = 1") - node.query("SELECT 2 SETTINGS use_query_cache = 1, query_cache_ttl = 1") - node.query("SELECT 3 SETTINGS use_query_cache = 1, query_cache_ttl = 1") time.sleep(2) - node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") - res = node.query( - "SELECT value FROM system.asynchronous_metrics WHERE metric = 'QueryCacheEntries'", - ) - assert res == "2\n" + # at this point, the query cache contains one entry and it is stale - # switch to a config with a maximum query cache size of 1 + res = node.query( + "SELECT count(*) FROM system.query_cache", + ) + assert res == "1\n" + + # switch to a config with a maximum query cache size of _0_ node.copy_file_to_container( - os.path.join(CONFIG_DIR, "smaller_query_cache.xml"), + os.path.join(CONFIG_DIR, "empty_query_cache.xml"), "/etc/clickhouse-server/config.d/default.xml", ) node.query("SYSTEM RELOAD CONFIG") - # check that eviction worked as expected - time.sleep(2) - node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") res = node.query( - "SELECT value FROM system.asynchronous_metrics WHERE metric = 'QueryCacheEntries'", - ) - assert ( - res == "2\n" - ) # "Why not 1?", you think. Reason is that QC uses the TTLCachePolicy that evicts lazily only upon insert. - # Not a real issue, can be changed later, at least there's a test now. - - # Also, you may also wonder "why query_cache_ttl = 1"? Reason is that TTLCachePolicy only removes *stale* entries. With the default TTL - # (60 sec), no entries would be removed at all. Again: not a real issue, can be changed later and there's at least a test now. - - # check that the new query cache maximum size is respected when more queries run - node.query("SELECT 4 SETTINGS use_query_cache = 1, query_cache_ttl = 1") - node.query("SELECT 5 SETTINGS use_query_cache = 1, query_cache_ttl = 1") - - time.sleep(2) - node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") - res = node.query( - "SELECT value FROM system.asynchronous_metrics WHERE metric = 'QueryCacheEntries'", + "SELECT count(*) FROM system.query_cache", ) assert res == "1\n" + # "Why not 0?", I hear you say. Reason is that QC uses the TTLCachePolicy that evicts lazily only upon insert. + # Not a real issue, can be changed later, at least there's a test now. + + # The next SELECT will find a single stale entry which is one entry too much according to the new config. + # This triggers the eviction of all stale entries, in this case the 'SELECT 1' result. + # Then, it tries to insert the 'SELECT 2' result but it also cannot be added according to the config. + node.query("SELECT 2 SETTINGS use_query_cache = 1, query_cache_ttl = 1") + res = node.query( + "SELECT count(*) FROM system.query_cache", + ) + assert res == "0\n" + + # The new maximum cache size is respected when more queries run + node.query("SELECT 3 SETTINGS use_query_cache = 1, query_cache_ttl = 1") + res = node.query( + "SELECT count(*) FROM system.query_cache", + ) + assert res == "0\n" # restore the original config node.copy_file_to_container( diff --git a/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.reference b/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.sql b/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.sql new file mode 100644 index 00000000000..d8de4facb38 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_asynchronous_metrics.sql @@ -0,0 +1,13 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SYSTEM DROP QUERY CACHE; + +-- Create an entry in the query cache +SELECT 1 SETTINGS use_query_cache = true; + +-- Asynchronous metrics must know about the entry +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT value FROM system.asynchronous_metrics WHERE metric = 'QueryCacheEntries'; + +SYSTEM DROP QUERY CACHE; From 919005c4f70b044ecd9cc1bbce5dc5e276e11929 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jul 2024 13:09:34 +0000 Subject: [PATCH 042/123] Incorporate review feedback --- .../test_runtime_configurable_cache_size/test.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_runtime_configurable_cache_size/test.py b/tests/integration/test_runtime_configurable_cache_size/test.py index 26676009e19..beaf83ea754 100644 --- a/tests/integration/test_runtime_configurable_cache_size/test.py +++ b/tests/integration/test_runtime_configurable_cache_size/test.py @@ -96,11 +96,11 @@ CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs") def test_query_cache_size_is_runtime_configurable(start_cluster): node.query("SYSTEM DROP QUERY CACHE") - # the initial config allows at most two query cache entries but we don't mind + # The initial config allows at most two query cache entries but we don't mind node.query("SELECT 1 SETTINGS use_query_cache = 1, query_cache_ttl = 1") time.sleep(2) - # at this point, the query cache contains one entry and it is stale + # At this point, the query cache contains one entry and it is stale res = node.query( "SELECT count(*) FROM system.query_cache", @@ -138,8 +138,17 @@ def test_query_cache_size_is_runtime_configurable(start_cluster): ) assert res == "0\n" - # restore the original config + # Restore the original config node.copy_file_to_container( os.path.join(CONFIG_DIR, "default.xml"), "/etc/clickhouse-server/config.d/default.xml", ) + + node.query("SYSTEM RELOAD CONFIG") + + # It is possible to insert entries again + node.query("SELECT 4 SETTINGS use_query_cache = 1, query_cache_ttl = 1") + res = node.query( + "SELECT count(*) FROM system.query_cache", + ) + assert res == "1\n" From 17b306dee522bf40b6020df5e664fcd9cc66dacf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 23 Jul 2024 16:20:12 +0200 Subject: [PATCH 043/123] fix tests with timeouts --- src/Common/tests/gtest_resolve_pool.cpp | 29 +++++++++++++++---------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/Common/tests/gtest_resolve_pool.cpp b/src/Common/tests/gtest_resolve_pool.cpp index b760b9b1524..c443e961cc7 100644 --- a/src/Common/tests/gtest_resolve_pool.cpp +++ b/src/Common/tests/gtest_resolve_pool.cpp @@ -33,7 +33,7 @@ size_t toMilliseconds(auto duration) return std::chrono::duration_cast(duration).count(); } -const auto epsilon = 500us; +const auto epsilon = 1ms; class ResolvePoolMock : public DB::HostResolver { @@ -358,53 +358,59 @@ void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses TEST_F(ResolvePoolTest, BannedForConsiquenceFail) { - auto history = 5ms; + auto history = 10ms; auto resolver = make_resolver(toMilliseconds(history)); auto failed_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*failed_addr)); - auto start_at = now(); failed_addr.setFail(); + auto start_at = now(); + ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); check_no_failed_address(1, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); sleep_until(start_at + history + epsilon); - start_at = now(); resolver->update(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(0, CurrentMetrics::get(metrics.banned_count)); failed_addr.setFail(); + start_at = now(); + check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); sleep_until(start_at + history + epsilon); - start_at = now(); resolver->update(); + + // too much time has passed + if (now() > start_at + 2*history - epsilon) + return; + ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); // ip still banned adter history_ms + update, because it was his second consiquent fail - check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); + check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + 2*history - epsilon); } TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail) { - auto history = 5ms; + auto history = 10ms; auto resolver = make_resolver(toMilliseconds(history)); auto failed_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*failed_addr)); - auto start_at = now(); + failed_addr.setFail(); + failed_addr.setFail(); + failed_addr.setFail(); - failed_addr.setFail(); - failed_addr.setFail(); - failed_addr.setFail(); + auto start_at = now(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); @@ -413,6 +419,7 @@ TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail) sleep_until(start_at + history + epsilon); resolver->update(); + // ip is cleared after just 1 history_ms interval. ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(0, CurrentMetrics::get(metrics.banned_count)); From 094eac641b0ff72e35b735a36eb5d70b6b52f3f1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 15:25:23 +0100 Subject: [PATCH 044/123] impl --- .../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 +++++++++++++++++++ 6 files changed, 120 insertions(+) create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference create mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.reference create mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.reference create mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference new file mode 100644 index 00000000000..0d51ecfac3b --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.reference @@ -0,0 +1,19 @@ +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 new file mode 100755 index 00000000000..aabba731816 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh @@ -0,0 +1,21 @@ +#!/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 new file mode 100644 index 00000000000..099b7574566 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.reference @@ -0,0 +1,19 @@ +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 new file mode 100755 index 00000000000..872f4c20a98 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh @@ -0,0 +1,21 @@ +#!/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 new file mode 100644 index 00000000000..35db4a22b4c --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.reference @@ -0,0 +1,19 @@ +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 new file mode 100755 index 00000000000..96276c96add --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh @@ -0,0 +1,21 @@ +#!/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 a7baafc0221178099ef4d683a1e4f6d81af160f2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 15:25:52 +0100 Subject: [PATCH 045/123] remove old --- .../03036_dynamic_read_subcolumns.reference | 57 ----------------- .../03036_dynamic_read_subcolumns.sh | 62 ------------------- 2 files changed, 119 deletions(-) delete mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference deleted file mode 100644 index 36984bc8b9b..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference +++ /dev/null @@ -1,57 +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 -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 -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.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh deleted file mode 100755 index 65517061b99..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ /dev/null @@ -1,62 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - - -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" -} - -$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;" - -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;" - -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 e671d4c55d520187f0fb2c5abb4ac301926d9651 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Jul 2024 14:43:49 +0000 Subject: [PATCH 046/123] more changes --- tests/integration/test_storage_s3/test.py | 137 ++++++++++++++-------- 1 file changed, 86 insertions(+), 51 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5453ad5a796..d13605170ec 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -962,9 +962,10 @@ def test_storage_s3_put_uncompressed(started_cluster): [pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz")], ) def test_storage_s3_put_gzip(started_cluster, extension, method): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] - filename = f"test_put_gzip.{extension}" + filename = f"{id}/test_put_gzip.{extension}" name = f"test_put_gzip_{extension}" data = [ "'Joseph Tomlinson',5", @@ -1001,6 +1002,7 @@ def test_storage_s3_put_gzip(started_cluster, extension, method): f = gzip.GzipFile(fileobj=buf, mode="rb") uncompressed_content = f.read().decode() assert sum([int(i.split(",")[1]) for i in uncompressed_content.splitlines()]) == 708 + run_query(instance, f"DROP TABLE {name}") def test_truncate_table(started_cluster): @@ -1026,14 +1028,24 @@ def test_truncate_table(started_cluster): len(list(minio.list_objects(started_cluster.minio_bucket, "truncate/"))) == 0 ): - return + break timeout -= 1 time.sleep(1) assert len(list(minio.list_objects(started_cluster.minio_bucket, "truncate/"))) == 0 - assert instance.query("SELECT * FROM {}".format(name)) == "" + # FIXME: there was a bug in test and it was never checked. + # Currently read from truncated table fails with + # DB::Exception: Failed to get object info: No response body.. + # HTTP response code: 404: while reading truncate: While executing S3Source + # assert instance.query("SELECT * FROM {}".format(name)) == "" + instance.query(f"DROP TABLE {name} SYNC") + assert ( + instance.query(f"SELECT count() FROM system.tables where name='{name}'") + == "0\n" + ) def test_predefined_connection_configuration(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances[ "dummy_without_named_collections" @@ -1061,7 +1073,9 @@ def test_predefined_connection_configuration(started_cluster): user="user", ) - instance.query(f"INSERT INTO {name} SELECT number FROM numbers(10)") + instance.query( + f"INSERT INTO {name} SELECT number FROM numbers(10) SETTINGS s3_truncate_on_insert=1" + ) result = instance.query(f"SELECT * FROM {name}") assert result == instance.query("SELECT number FROM numbers(10)") @@ -1075,9 +1089,11 @@ def test_predefined_connection_configuration(started_cluster): "To execute this query, it's necessary to have the grant NAMED COLLECTION ON no_collection" in error ) - instance = started_cluster.instances["dummy"] # has named collection access - error = instance.query_and_get_error("SELECT * FROM s3(no_collection)") + instance2 = started_cluster.instances["dummy"] # has named collection access + error = instance2.query_and_get_error("SELECT * FROM s3(no_collection)") assert "There is no named collection `no_collection`" in error + instance.query("DROP USER user") + instance.query(f"DROP TABLE {name}") result = "" @@ -1227,7 +1243,7 @@ def test_s3_schema_inference(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into table function s3(s3_native, structure='a Int32, b String', format='Native') select number, randomString(100) from numbers(5000000)" + f"insert into table function s3(s3_native, structure='a Int32, b String', format='Native') select number, randomString(100) from numbers(5000000) SETTINGS s3_truncate_on_insert=1" ) result = instance.query(f"desc s3(s3_native, format='Native')") assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" @@ -1267,6 +1283,9 @@ def test_s3_schema_inference(started_cluster): result = instance.query(f"select count(*) from {table_function}") assert int(result) == 5000000 + instance.query("drop table schema_inference") + instance.query("drop table schema_inference_2") + def test_empty_file(started_cluster): bucket = started_cluster.minio_bucket @@ -1302,6 +1321,7 @@ def test_overwrite(started_cluster): result = instance.query(f"select count() from test_overwrite") assert int(result) == 200 + instance.query(f"drop table test_overwrite") def test_create_new_files_on_insert(started_cluster): @@ -1343,6 +1363,7 @@ def test_create_new_files_on_insert(started_cluster): result = instance.query(f"select count() from test_multiple_inserts") assert int(result) == 60 + instance.query("drop table test_multiple_inserts") def test_format_detection(started_cluster): @@ -1350,7 +1371,9 @@ def test_format_detection(started_cluster): instance = started_cluster.instances["dummy"] instance.query(f"create table arrow_table_s3 (x UInt64) engine=S3(s3_arrow)") - instance.query(f"insert into arrow_table_s3 select 1") + instance.query( + f"insert into arrow_table_s3 select 1 settings s3_truncate_on_insert=1" + ) result = instance.query(f"select * from s3(s3_arrow)") assert int(result) == 1 @@ -1365,7 +1388,9 @@ def test_format_detection(started_cluster): assert int(result) == 1 instance.query(f"create table parquet_table_s3 (x UInt64) engine=S3(s3_parquet2)") - instance.query(f"insert into parquet_table_s3 select 1") + instance.query( + f"insert into parquet_table_s3 select 1 settings s3_truncate_on_insert=1" + ) result = instance.query(f"select * from s3(s3_parquet2)") assert int(result) == 1 @@ -1378,64 +1403,67 @@ def test_format_detection(started_cluster): f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test.parquet')" ) assert int(result) == 1 + instance.query(f"drop table arrow_table_s3") + instance.query(f"drop table parquet_table_s3") def test_schema_inference_from_globs(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] instance.query( - f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) instance.query( - f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" + f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" ) url_filename = "test{1,2}.jsoncompacteachrow" result = instance.query( - f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}') settings input_format_json_infer_incomplete_types_as_strings=0" + f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{url_filename}') settings input_format_json_infer_incomplete_types_as_strings=0" ) assert result.strip() == "c1\tNullable(Int64)" result = instance.query( - f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}') settings input_format_json_infer_incomplete_types_as_strings=0" + f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{url_filename}') settings input_format_json_infer_incomplete_types_as_strings=0" ) assert sorted(result.split()) == ["0", "\\N"] result = instance.query( - f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" + f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" ) assert result.strip() == "c1\tNullable(Int64)" result = instance.query( - f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" + f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" ) assert sorted(result.split()) == ["0", "\\N"] instance.query( - f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) url_filename = "test{1,3}.jsoncompacteachrow" result = instance.query_and_get_error( - f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}') settings schema_inference_use_cache_for_s3=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{url_filename}') settings schema_inference_use_cache_for_s3=0, input_format_json_infer_incomplete_types_as_strings=0" ) assert "All attempts to extract table structure from files failed" in result result = instance.query_and_get_error( - f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}') settings schema_inference_use_cache_for_url=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{url_filename}') settings schema_inference_use_cache_for_url=0, input_format_json_infer_incomplete_types_as_strings=0" ) assert "All attempts to extract table structure from files failed" in result instance.query( - f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" + f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" ) result = instance.query_and_get_error( - f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test*.jsoncompacteachrow') settings schema_inference_use_cache_for_s3=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test*.jsoncompacteachrow') settings schema_inference_use_cache_for_s3=0, input_format_json_infer_incomplete_types_as_strings=0" ) assert "CANNOT_EXTRACT_TABLE_STRUCTURE" in result @@ -1443,7 +1471,7 @@ def test_schema_inference_from_globs(started_cluster): url_filename = "test{0,1,2,3}.jsoncompacteachrow" result = instance.query_and_get_error( - f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}') settings schema_inference_use_cache_for_url=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/{url_filename}') settings schema_inference_use_cache_for_url=0, input_format_json_infer_incomplete_types_as_strings=0" ) assert "CANNOT_EXTRACT_TABLE_STRUCTURE" in result @@ -1503,9 +1531,12 @@ def test_signatures(started_cluster): ) assert "S3_ERROR" in error + instance.query(f"drop table test_signatures") + def test_select_columns(started_cluster): bucket = started_cluster.minio_bucket + id = uuid.uuid4() instance = started_cluster.instances["dummy"] name = "test_table2" structure = "id UInt32, value1 Int32, value2 Int32" @@ -1519,36 +1550,37 @@ def test_select_columns(started_cluster): instance.query( f"INSERT INTO {name} SELECT * FROM generateRandom('{structure}') LIMIT {limit} SETTINGS s3_truncate_on_insert=1" ) - instance.query(f"SELECT value2 FROM {name}") + instance.query(f"SELECT value2, '{id}' FROM {name}") instance.query("SYSTEM FLUSH LOGS") result1 = instance.query( - f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT value2 FROM {name}'" + f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT value2, ''{id}'' FROM {name}'" ) - instance.query(f"SELECT * FROM {name}") + instance.query(f"SELECT *, '{id}' FROM {name}") instance.query("SYSTEM FLUSH LOGS") result2 = instance.query( - f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT * FROM {name}'" + f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT *, ''{id}'' FROM {name}'" ) assert round(int(result2) / int(result1)) == 3 def test_insert_select_schema_inference(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native') select toUInt64(1) as x" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test_insert_select.native') select toUInt64(1) as x" ) result = instance.query( - f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native')" + f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test_insert_select.native')" ) assert result.strip() == "x\tUInt64" result = instance.query( - f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native')" + f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{id}/test_insert_select.native')" ) assert int(result) == 1 @@ -1558,7 +1590,7 @@ def test_parallel_reading_with_memory_limit(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') select * from numbers(1000000)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') select * from numbers(1000000) SETTINGS s3_truncate_on_insert=1" ) result = instance.query_and_get_error( @@ -1579,7 +1611,7 @@ def test_wrong_format_usage(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_wrong_format.native') select * from numbers(10e6)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_wrong_format.native') select * from numbers(10e6) SETTINGS s3_truncate_on_insert=1" ) # size(test_wrong_format.native) = 10e6*8+16(header) ~= 76MiB @@ -2102,11 +2134,11 @@ def test_read_subcolumns(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS s3_truncate_on_insert=1" ) res = instance.query( @@ -2165,7 +2197,7 @@ def test_read_subcolumn_time(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32') select (42) SETTINGS s3_truncate_on_insert=1" ) res = instance.query( @@ -2176,29 +2208,30 @@ def test_read_subcolumn_time(started_cluster): def test_filtering_by_file_or_path(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter1.tsv', auto, 'x UInt64') select 1" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter1.tsv', auto, 'x UInt64') select 1 SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter2.tsv', auto, 'x UInt64') select 2" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter2.tsv', auto, 'x UInt64') select 2 SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter3.tsv', auto, 'x UInt64') select 3" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter3.tsv', auto, 'x UInt64') select 3 SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter*.tsv') where _file = 'test_filter1.tsv'" + f"select count(), '{id}' from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter*.tsv') where _file = 'test_filter1.tsv'" ) instance.query("SYSTEM FLUSH LOGS") result = instance.query( - f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query like '%select%s3%test_filter%' AND type='QueryFinish'" + f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query like '%{id}%' AND type='QueryFinish'" ) assert int(result) == 1 @@ -2211,54 +2244,56 @@ def test_filtering_by_file_or_path(started_cluster): def test_union_schema_inference_mode(started_cluster): + id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["s3_non_default"] + file_name_prefix = f"test_union_schema_inference_{id}_" instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference1.jsonl') select 1 as a" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}1.jsonl') select 1 as a SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference2.jsonl') select 2 as b" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}2.jsonl') select 2 as b SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference3.jsonl') select 2 as c" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}3.jsonl') select 2 as c SETTINGS s3_truncate_on_insert=1" ) instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference4.jsonl', TSV) select 'Error'" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}4.jsonl', TSV) select 'Error' SETTINGS s3_truncate_on_insert=1" ) for engine in ["s3", "url"]: instance.query("system drop schema cache for s3") result = instance.query( - f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\nc\tNullable(Int64)\n" result = instance.query( - "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + f"select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%{file_name_prefix}%' order by file format TSV" ) assert ( - result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" - "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" - "UNION\ttest_union_schema_inference3.jsonl\tc Nullable(Int64)\n" + result == f"UNION\t{file_name_prefix}1.jsonl\ta Nullable(Int64)\n" + f"UNION\t{file_name_prefix}2.jsonl\tb Nullable(Int64)\n" + f"UNION\t{file_name_prefix}3.jsonl\tc Nullable(Int64)\n" ) result = instance.query( - f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}{{1,2,3}}.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "1\t\\N\t\\N\n" "\\N\t2\t\\N\n" "\\N\t\\N\t2\n" instance.query(f"system drop schema cache for {engine}") result = instance.query( - f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "b\tNullable(Int64)\n" result = instance.query( - f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert ( result == "a\tNullable(Int64)\n" @@ -2267,7 +2302,7 @@ def test_union_schema_inference_mode(started_cluster): ) error = instance.query_and_get_error( - f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3,4}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{file_name_prefix}{{1,2,3,4}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert "CANNOT_EXTRACT_TABLE_STRUCTURE" in error From bc9c462155bed43eeb72415660917900ee4e7b58 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 15:47:55 +0100 Subject: [PATCH 047/123] more comments --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0e713150625..46482bc0959 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -61,6 +61,9 @@ static size_t calculateMinMarksPerTask( const size_t part_marks_count = part.getMarksCount(); if (part_marks_count && part.data_part->isStoredOnRemoteDisk()) { + /// We assume that most of the time prewhere does it's job good meaning that lion's share of the rows is filtered out. + /// Which means in turn that for most of the rows we will read only the columns from prewhere clause. + /// So it makes sense to use only them for the estimation. const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : columns_to_read; @@ -69,6 +72,7 @@ static size_t calculateMinMarksPerTask( const auto avg_mark_bytes = std::max(part_compressed_bytes / part_marks_count, 1); const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + /// We also create at least two tasks per thread to have something to steal from a slow thread. const auto heuristic_min_marks = std::min(pool_settings.sum_marks / pool_settings.threads / 2, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_per_task) From e0a7aa80af298465292a5227e65fe21c931b1b9b Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Jul 2024 16:57:07 +0200 Subject: [PATCH 048/123] Enable repeat for flaky check for more tests hardening --- tests/ci/integration_tests_runner.py | 18 ++++++++++++++---- tests/integration/runner | 6 +++++- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 21f16d995a4..2b348be8b51 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -29,7 +29,8 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 10 +FLAKY_TRIES_COUNT = 10 # run whole pytest several times +FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 MAX_TIME_IN_SANDBOX = 20 * 60 # 20 minutes @@ -568,6 +569,7 @@ class ClickhouseIntegrationTestsRunner: tests_in_group, num_tries, num_workers, + repeat_count, ): try: return self.run_test_group( @@ -576,6 +578,7 @@ class ClickhouseIntegrationTestsRunner: tests_in_group, num_tries, num_workers, + repeat_count, ) except Exception as e: logging.info("Failed to run %s:\n%s", test_group, e) @@ -598,6 +601,7 @@ class ClickhouseIntegrationTestsRunner: tests_in_group, num_tries, num_workers, + repeat_count, ): counters = { "ERROR": [], @@ -639,6 +643,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = " ".join([shlex.quote(test) for test in sorted(test_names)]) parallel_cmd = f" --parallel {num_workers} " if num_workers > 0 else "" + repeat_cmd = f" --count {repeat_count} " if repeat_count > 0 else "" # -r -- show extra test summary: # -f -- (f)ailed # -E -- (E)rror @@ -647,7 +652,7 @@ class ClickhouseIntegrationTestsRunner: cmd = ( f"cd {repo_path}/tests/integration && " f"timeout --signal=KILL 1h ./runner {self._get_runner_opts()} " - f"{image_cmd} -t {test_cmd} {parallel_cmd} -- -rfEps --run-id={i} " + f"{image_cmd} -t {test_cmd} {parallel_cmd} {repeat_cmd} -- -rfEps --run-id={i} " f"--color=no --durations=0 {_get_deselect_option(self.should_skip_tests())} " f"| tee {info_path}" ) @@ -784,7 +789,12 @@ class ClickhouseIntegrationTestsRunner: final_retry += 1 logging.info("Running tests for the %s time", i) counters, tests_times, log_paths = self.try_run_test_group( - repo_path, "bugfix" if should_fail else "flaky", tests_to_run, 1, 1 + repo_path, + "bugfix" if should_fail else "flaky", + tests_to_run, + 1, + 1, + FLAKY_REPEAT_COUNT, ) logs += log_paths if counters["FAILED"]: @@ -919,7 +929,7 @@ class ClickhouseIntegrationTestsRunner: for group, tests in items_to_run: logging.info("Running test group %s containing %s tests", group, len(tests)) group_counters, group_test_times, log_paths = self.try_run_test_group( - repo_path, group, tests, MAX_RETRY, NUM_WORKERS + repo_path, group, tests, MAX_RETRY, NUM_WORKERS, 0 ) total_tests = 0 for counter, value in group_counters.items(): diff --git a/tests/integration/runner b/tests/integration/runner index a583d7fe897..fc13cb9807a 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -243,6 +243,8 @@ if __name__ == "__main__": "-n", "--parallel", action="store", dest="parallel", help="Parallelism" ) + parser.add_argument("--count", action="store", dest="count", help="Repeat count") + parser.add_argument( "--no-random", action="store", @@ -318,6 +320,8 @@ if __name__ == "__main__": parallel_args += "--dist=loadfile" parallel_args += f" -n {args.parallel}".format() + repeat_args = f" --count {args.count}" if args.count > 0 else "" + rand_args = "" # if not args.no_random: # rand_args += f"--random-seed={os.getpid()}" @@ -409,7 +413,7 @@ if __name__ == "__main__": f"--volume={args.utils_dir}/grpc-client/pb2:/ClickHouse/utils/grpc-client/pb2 " f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " f"-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 {use_old_analyzer} -e PYTHONUNBUFFERED=1 " - f'-e PYTEST_ADDOPTS="{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv"' + f'-e PYTEST_ADDOPTS="{parallel_args} {repeat_args} {pytest_opts} {tests_list} {rand_args} -vvv"' f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}" ) From 2f8b28a17cd885581b4cb177643b952d7975d30c Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Jul 2024 17:05:23 +0200 Subject: [PATCH 049/123] Touch test to trigger flaky check --- tests/integration/test_backup_restore_new/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index d8662fad011..fbdc64725d5 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,8 +1,8 @@ -import pytest import glob -import re -import random import os.path +import pytest +import random +import re import sys from collections import namedtuple from helpers.cluster import ClickHouseCluster From c7400fbd508af022f9fdace728094444e03c1d28 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Jul 2024 18:08:07 +0200 Subject: [PATCH 050/123] Add more diagnostics in case of digest mismatch for DatabaseReplicated --- src/Databases/DatabaseReplicated.cpp | 249 +++++++++++++++++++-------- src/Databases/DatabaseReplicated.h | 10 +- 2 files changed, 185 insertions(+), 74 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 4c079ae5300..d0648faf07b 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -671,92 +671,96 @@ void DatabaseReplicated::stopLoading() DatabaseAtomic::stopLoading(); } -bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const +void DatabaseReplicated::dumpLocalTablesForDebugOnly(const ContextPtr & local_context) const { - if (debug_check) + auto table_names = getAllTableNames(context.lock()); + for (const auto & table_name : table_names) { - /// Reduce number of debug checks - if (thread_local_rng() % 16) - return true; + auto ast_ptr = tryGetCreateTableQuery(table_name, local_context); + if (ast_ptr) + LOG_DEBUG(log, "[local] Table {} create query is {}", table_name, queryToString(ast_ptr)); + else + LOG_DEBUG(log, "[local] Table {} has no create query", table_name); } - - LOG_TEST(log, "Current in-memory metadata digest: {}", tables_metadata_digest); - - /// Database is probably being dropped - if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) - return true; - - UInt64 local_digest = 0; - { - std::lock_guard lock{mutex}; - for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) - local_digest += getMetadataHash(table.first); - } - - if (local_digest != tables_metadata_digest) - { - LOG_ERROR(log, "Digest of local metadata ({}) is not equal to in-memory digest ({})", local_digest, tables_metadata_digest); - return false; - } - - /// Do not check digest in Keeper after internal subquery, it's probably not committed yet - if (local_context->isInternalSubquery()) - return true; - - /// Check does not make sense to check digest in Keeper during recovering - if (is_recovering) - return true; - - String zk_digest = getZooKeeper()->get(replica_path + "/digest"); - String local_digest_str = toString(local_digest); - if (zk_digest != local_digest_str) - { - LOG_ERROR(log, "Digest of local metadata ({}) is not equal to digest in Keeper ({})", local_digest_str, zk_digest); - return false; - } - - return true; } -void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_context) const +void DatabaseReplicated::dumpTablesInZooKeeperForDebugOnly() const { - /// Replicas will set correct name of current database in query context (database name can be different on replicas) - if (auto * ddl_query = dynamic_cast(query.get())) + UInt32 max_log_ptr; + auto table_name_to_metadata = tryGetConsistentMetadataSnapshot(getZooKeeper(), max_log_ptr); + for (const auto & [table_name, create_table_query] : table_name_to_metadata) { - if (ddl_query->getDatabase() != getDatabaseName()) - throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed"); - ddl_query->database.reset(); - - if (auto * create = query->as()) + auto query_ast = parseQueryFromMetadataInZooKeeper(table_name, create_table_query); + if (query_ast) { - if (create->storage) - checkTableEngine(*create, *create->storage, query_context); + LOG_DEBUG(log, "[zookeeper] Table {} create query is {}", table_name, queryToString(query_ast)); + } + else + { + LOG_DEBUG(log, "[zookeeper] Table {} has no create query", table_name); + } + } +} - if (create->targets) +void DatabaseReplicated::tryCompareLocalAndZooKeeperTablesAndDumpDiffForDebugOnly(const ContextPtr & local_context) const +{ + UInt32 max_log_ptr; + auto table_name_to_metadata_in_zk = tryGetConsistentMetadataSnapshot(getZooKeeper(), max_log_ptr); + auto table_names_local = getAllTableNames(local_context); + + if (table_name_to_metadata_in_zk.size() != table_names_local.size()) + LOG_DEBUG(log, "Amount of tables in zk {} locally {}", table_name_to_metadata_in_zk.size(), table_names_local.size()); + + std::unordered_set checked_tables; + + for (const auto & table_name : table_names_local) + { + auto local_ast_ptr = tryGetCreateTableQuery(table_name, local_context); + if (table_name_to_metadata_in_zk.contains(table_name)) + { + checked_tables.insert(table_name); + auto create_table_query_in_zk = table_name_to_metadata_in_zk[table_name]; + auto zk_ast_ptr = parseQueryFromMetadataInZooKeeper(table_name, create_table_query_in_zk); + + if (local_ast_ptr == nullptr && zk_ast_ptr == nullptr) { - for (const auto & inner_table_engine : create->targets->getInnerEngines()) - checkTableEngine(*create, *inner_table_engine, query_context); + LOG_DEBUG(log, "AST for table {} is the same (nullptr) in local and ZK", table_name); + } + else if (local_ast_ptr != nullptr && zk_ast_ptr != nullptr && queryToString(local_ast_ptr) != queryToString(zk_ast_ptr)) + { + LOG_DEBUG(log, "AST differs for table {}, local {}, in zookeeper {}", table_name, queryToString(local_ast_ptr), queryToString(zk_ast_ptr)); + } + else if (local_ast_ptr == nullptr) + { + LOG_DEBUG(log, "AST differs for table {}, local nullptr, in zookeeper {}", table_name, queryToString(zk_ast_ptr)); + } + else if (zk_ast_ptr == nullptr) + { + LOG_DEBUG(log, "AST differs for table {}, local {}, in zookeeper nullptr", table_name, queryToString(local_ast_ptr)); + } + else + { + LOG_DEBUG(log, "AST for table {} is the same in local and ZK", table_name); } } - } - - if (const auto * query_alter = query->as()) - { - for (const auto & command : query_alter->command_list->children) + else { - if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as().type)) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query"); + if (local_ast_ptr == nullptr) + LOG_DEBUG(log, "Table {} exists locally, but missing in ZK", table_name); + else + LOG_DEBUG(log, "Table {} exists locally with AST {}, but missing in ZK", table_name, queryToString(local_ast_ptr)); } } - - if (auto * query_drop = query->as()) + for (const auto & [table_name, table_metadata] : table_name_to_metadata_in_zk) { - if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context->getSettingsRef().database_replicated_always_detach_permanently) - query_drop->permanently = true; - if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently) - throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. " - "Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA or set " - "database_replicated_always_detach_permanently to 1"); + if (!checked_tables.contains(table_name)) + { + auto zk_ast_ptr = parseQueryFromMetadataInZooKeeper(table_name, table_metadata); + if (zk_ast_ptr == nullptr) + LOG_DEBUG(log, "Table {} exists in ZK with AST {}, but missing locally", table_name, queryToString(zk_ast_ptr)); + else + LOG_DEBUG(log, "Table {} exists in ZK, but missing locally", table_name); + } } } @@ -839,6 +843,107 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora "to distinguish different shards and replicas"); } +bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const +{ + if (debug_check) + { + /// Reduce number of debug checks + if (thread_local_rng() % 16) + return true; + } + + LOG_TEST(log, "Current in-memory metadata digest: {}", tables_metadata_digest); + + /// Database is probably being dropped + if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) + return true; + + UInt64 local_digest = 0; + { + std::lock_guard lock{mutex}; + for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) + local_digest += getMetadataHash(table.first); + } + + if (local_digest != tables_metadata_digest) + { + LOG_ERROR(log, "Digest of local metadata ({}) is not equal to in-memory digest ({})", local_digest, tables_metadata_digest); + +#ifndef NDEBUG + dumpLocalTablesForDebugOnly(local_context); + dumpTablesInZooKeeperForDebugOnly(); + tryCompareLocalAndZooKeeperTablesAndDumpDiffForDebugOnly(local_context); +#endif + + return false; + } + + /// Do not check digest in Keeper after internal subquery, it's probably not committed yet + if (local_context->isInternalSubquery()) + return true; + + /// Check does not make sense to check digest in Keeper during recovering + if (is_recovering) + return true; + + String zk_digest = getZooKeeper()->get(replica_path + "/digest"); + String local_digest_str = toString(local_digest); + if (zk_digest != local_digest_str) + { + LOG_ERROR(log, "Digest of local metadata ({}) is not equal to digest in Keeper ({})", local_digest_str, zk_digest); +#ifndef NDEBUG + dumpLocalTablesForDebugOnly(local_context); + dumpTablesInZooKeeperForDebugOnly(); + tryCompareLocalAndZooKeeperTablesAndDumpDiffForDebugOnly(local_context); +#endif + return false; + } + + return true; +} + +void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_context) const +{ + /// Replicas will set correct name of current database in query context (database name can be different on replicas) + if (auto * ddl_query = dynamic_cast(query.get())) + { + if (ddl_query->getDatabase() != getDatabaseName()) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed"); + ddl_query->database.reset(); + + if (auto * create = query->as()) + { + if (create->storage) + checkTableEngine(*create, *create->storage, query_context); + + if (create->targets) + { + for (const auto & inner_table_engine : create->targets->getInnerEngines()) + checkTableEngine(*create, *inner_table_engine, query_context); + } + } + } + + if (const auto * query_alter = query->as()) + { + for (const auto & command : query_alter->command_list->children) + { + if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as().type)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query"); + } + } + + if (auto * query_drop = query->as()) + { + if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context->getSettingsRef().database_replicated_always_detach_permanently) + query_drop->permanently = true; + if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently) + throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. " + "Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA or set " + "database_replicated_always_detach_permanently to 1"); + } +} + BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, QueryFlags flags) { waitDatabaseStarted(); @@ -1253,7 +1358,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep current_zookeeper->set(replica_path + "/digest", toString(tables_metadata_digest)); } -std::map DatabaseReplicated::tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr) +std::map DatabaseReplicated::tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr) const { return getConsistentMetadataSnapshotImpl(zookeeper, {}, /* max_retries= */ 10, max_log_ptr); } @@ -1314,7 +1419,7 @@ std::map DatabaseReplicated::getConsistentMetadataSnapshotImpl( return table_name_to_metadata; } -ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query) +ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query) const { ParserCreateQuery parser; String description = "in ZooKeeper " + zookeeper_path + "/metadata/" + node_name; diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8c3fa7c87f6..2d96695f7cc 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -109,14 +109,15 @@ private: void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const; void checkTableEngine(const ASTCreateQuery & query, ASTStorage & storage, ContextPtr query_context) const; + void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 & max_log_ptr); - std::map tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr); + std::map tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr) const; std::map getConsistentMetadataSnapshotImpl(const ZooKeeperPtr & zookeeper, const FilterByNameFunction & filter_by_table_name, size_t max_retries, UInt32 & max_log_ptr) const; - ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query); + ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query) const; String readMetadataFile(const String & table_name) const; ClusterPtr getClusterImpl(bool all_groups = false) const; @@ -132,6 +133,11 @@ private: UInt64 getMetadataHash(const String & table_name) const; bool checkDigestValid(const ContextPtr & local_context, bool debug_check = true) const TSA_REQUIRES(metadata_mutex); + /// For debug purposes only, don't use in production code + void dumpLocalTablesForDebugOnly(const ContextPtr & local_context) const; + void dumpTablesInZooKeeperForDebugOnly() const; + void tryCompareLocalAndZooKeeperTablesAndDumpDiffForDebugOnly(const ContextPtr & local_context) const; + void waitDatabaseStarted() const override; void stopLoading() override; From 7a65eb08ed571037206640cbdfe305a383e52c9b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 18:00:33 +0100 Subject: [PATCH 051/123] add forgotten file --- .../03036_dynamic_read_subcolumns.lib | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib new file mode 100755 index 00000000000..4914051db82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.lib @@ -0,0 +1,35 @@ +#!/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" +} From 3df8faf640945ca73a292fe8da6c8b4eed6398b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Jul 2024 19:07:19 +0200 Subject: [PATCH 052/123] Add initial 24.7 changelog --- CHANGELOG.md | 179 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 179 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e2eb65e2967..4fddd7d7685 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v24.7, 2024-07-30](#247)**
**[ClickHouse release v24.6, 2024-07-01](#246)**
**[ClickHouse release v24.5, 2024-05-30](#245)**
**[ClickHouse release v24.4, 2024-04-30](#244)**
@@ -9,6 +10,184 @@ # 2024 Changelog +### ClickHouse release 24.7, 2024-07-30 + +#### Backward Incompatible Change +* Forbid `CRATE MATERIALIZED VIEW ... ENGINE Replicated*MergeTree POPULATE AS SELECT ...` with Replicated databases. [#63963](https://github.com/ClickHouse/ClickHouse/pull/63963) ([vdimir](https://github.com/vdimir)). +* `clickhouse-keeper-client` will only accept paths in string literals, such as `ls '/hello/world'`, not bare strings such as `ls /hello/world`. [#65494](https://github.com/ClickHouse/ClickHouse/pull/65494) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Extend function `tuple` to construct named tuples in query. Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). +* Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). +* A new table function, `fuzzQuery,` was added. This function allows you to modify a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1');`. [#62103](https://github.com/ClickHouse/ClickHouse/pull/62103) ([pufit](https://github.com/pufit)). +* Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). +* Support JWT authentication in `clickhouse-client`. [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). +* Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). +* Support accept_invalid_certificate in client's config in order to allow for client to connect over secure TCP to a server running with self-signed certificate - can be used as a shorthand for corresponding `openSSL` client settings `verificationMode=none` + `invalidCertificateHandler.name=AcceptCertificateHandler`. [#65238](https://github.com/ClickHouse/ClickHouse/pull/65238) ([peacewalker122](https://github.com/peacewalker122)). +* Add system.error_log which contains history of error values from table system.errors, periodically flushed to disk. [#65381](https://github.com/ClickHouse/ClickHouse/pull/65381) ([Pablo Marcos](https://github.com/pamarcos)). +* Add aggregate function `groupConcat`. About the same as `arrayStringConcat( groupArray(column), ',')` Can receive 2 parameters: a string delimiter and the number of elements to be processed. [#65451](https://github.com/ClickHouse/ClickHouse/pull/65451) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Add AzureQueue storage. [#65458](https://github.com/ClickHouse/ClickHouse/pull/65458) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). +* Allow system administrators to configure `logger.console_log_level`. [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). +* Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). +* Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). + +#### Experimental Feature +* Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). +* Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). +* Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Performance Improvement +* Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). +* Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). +* Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). +* Improve performance of loading `plain_rewritable` metadata. [#65634](https://github.com/ClickHouse/ClickHouse/pull/65634) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attaching tables on read-only disks will use fewer resources by not loading outdated parts. [#65635](https://github.com/ClickHouse/ClickHouse/pull/65635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). +* Unload primary index of outdated parts to reduce total memory usage. [#65852](https://github.com/ClickHouse/ClickHouse/pull/65852) ([Anton Popov](https://github.com/CurtizJ)). +* Functions `replaceRegexpAll` and `replaceRegexpOne` are now significantly faster if the pattern is trivial, i.e. contains no metacharacters, pattern classes, flags, grouping characters etc. (Thanks to Taiyang Li). [#66185](https://github.com/ClickHouse/ClickHouse/pull/66185) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Improvement +* The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). +* The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). +* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). +* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_ignore_key_case`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). +* In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Make an interactive client for clickhouse-disks, add local disk from the local directory. [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). +* Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). +* `StorageS3Queue` related fixes and improvements. Deduce a default value of `s3queue_processing_threads_num` according to the number of physical cpu cores on the server (instead of the previous default value as 1). Set default value of `s3queue_loading_retries` to 10. Fix possible vague "Uncaught exception" in exception column of `system.s3queue`. Do not increment retry count on `MEMORY_LIMIT_EXCEEDED` exception. Move files commit to a stage after insertion into table fully finished to avoid files being commited while not inserted. Add settings `s3queue_max_processed_files_before_commit`, `s3queue_max_processed_rows_before_commit`, `s3queue_max_processed_bytes_before_commit`, `s3queue_max_processing_time_sec_before_commit`, to better control commit and flush time. [#65046](https://github.com/ClickHouse/ClickHouse/pull/65046) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed broken multiple columns aggregation on s390x. [#65062](https://github.com/ClickHouse/ClickHouse/pull/65062) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). +* s3 requests: Reduce retry time for queries, increase retries count for backups. 8.5 minutes and 100 retires for queries, 1.2 hours and 1000 retries for backup restore. [#65232](https://github.com/ClickHouse/ClickHouse/pull/65232) ([Sema Checherinda](https://github.com/CheSema)). +* Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). +* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). +* Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). +* Support query plan LIMIT optimization. Support LIMIT pushdown for PostgreSQL storage and table function. [#65454](https://github.com/ClickHouse/ClickHouse/pull/65454) ([Maksim Kita](https://github.com/kitaisreal)). +* `arrayMin`/`arrayMax` can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). +* Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). +* Do not create format settings for each row when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). +* Fixed out-of-range exception in parsing Dwarf5 on s390x. [#65501](https://github.com/ClickHouse/ClickHouse/pull/65501) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Reduce `clickhouse-local` prompt to just `:)`. `getFQDNOrHostName()` takes too long on macOS, and we don't want a hostname in the prompt for `clickhouse-local` anyway. [#65510](https://github.com/ClickHouse/ClickHouse/pull/65510) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Avoid printing a message from jemalloc about per-CPU arenas on low-end virtual machines. [#65532](https://github.com/ClickHouse/ClickHouse/pull/65532) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new option to config `` which allow to specify how often clickhouse will reload config. [#65545](https://github.com/ClickHouse/ClickHouse/pull/65545) ([alesapin](https://github.com/alesapin)). +* Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). +* When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). +* Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). +* `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a script to backup your files to ClickHouse. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* PostgreSQL source support cancel. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). +* Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow to use `concat` function with empty arguments ``` sql :) select concat();. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). +* Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). +* Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). +* DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). +* Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). +* Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). +* Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). +* Add settings to ignore ON CLUSTER clause in queries for named collection management with replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Function `generateSnowflakeID` now allows to specify a machine ID as a parameter to prevent collisions in large clusters. [#66374](https://github.com/ClickHouse/ClickHouse/pull/66374) ([ZAWA_ll](https://github.com/Zawa-ll)). +* Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Fix the VALID UNTIL clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). +* Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug with cancelation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Fix unexpected projection name when query with CTE. [#65267](https://github.com/ClickHouse/ClickHouse/pull/65267) ([wudidapaopao](https://github.com/wudidapaopao)). +* Require `dictGet` privilege when accessing dictionaries via direct query or the `Dictionary` table engine. [#65359](https://github.com/ClickHouse/ClickHouse/pull/65359) ([Joe Lynch](https://github.com/joelynch)). +* Fix user-specific S3 auth with incremental backups. [#65481](https://github.com/ClickHouse/ClickHouse/pull/65481) ([Antonio Andelic](https://github.com/antonio2368)). +* Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix getting exception `Index out of bound for blob metadata` in case all files from list batch were filtered out. [#65523](https://github.com/ClickHouse/ClickHouse/pull/65523) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix NOT_FOUND_COLUMN_IN_BLOCK for deduplicate merge of projection. [#65573](https://github.com/ClickHouse/ClickHouse/pull/65573) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed a bug that compatibility level '23.4' was not properly applied. [#65737](https://github.com/ClickHouse/ClickHouse/pull/65737) ([cw5121](https://github.com/cw5121)). +* Fix odbc table with nullable fields. [#65738](https://github.com/ClickHouse/ClickHouse/pull/65738) ([Rodolphe Dugé de Bernonville](https://github.com/RodolpheDuge)). +* Fix data race in `TCPHandler`, which could happen on fatal error. [#65744](https://github.com/ClickHouse/ClickHouse/pull/65744) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Fix a bug leads to EmbeddedRocksDB with TTL write corrupted SST files. [#65816](https://github.com/ClickHouse/ClickHouse/pull/65816) ([Duc Canh Le](https://github.com/canhld94)). +* Functions `bitTest`, `bitTestAll`, and `bitTestAny` now return an error if the specified bit index is out-of-bounds [#65818](https://github.com/ClickHouse/ClickHouse/pull/65818) ([Pablo Marcos](https://github.com/pamarcos)). +* Setting `join_any_take_last_row` is supported in any query with hash join. [#65820](https://github.com/ClickHouse/ClickHouse/pull/65820) ([vdimir](https://github.com/vdimir)). +* Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Functions `bitShiftLeft` and `bitShitfRight` return an error for out of bounds shift positions [#65838](https://github.com/ClickHouse/ClickHouse/pull/65838) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix growing memory usage in S3Queue. [#65839](https://github.com/ClickHouse/ClickHouse/pull/65839) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix tie handling in `arrayAUC` to match sklearn. [#65840](https://github.com/ClickHouse/ClickHouse/pull/65840) ([gabrielmcg44](https://github.com/gabrielmcg44)). +* Fix possible issues with MySQL server protocol TLS connections. [#65917](https://github.com/ClickHouse/ClickHouse/pull/65917) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible issues with MySQL client protocol TLS connections. [#65938](https://github.com/ClickHouse/ClickHouse/pull/65938) ([Azat Khuzhin](https://github.com/azat)). +* Fix handling of `SSL_ERROR_WANT_READ`/`SSL_ERROR_WANT_WRITE` with zero timeout. [#65941](https://github.com/ClickHouse/ClickHouse/pull/65941) ([Azat Khuzhin](https://github.com/azat)). +* Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Column _size in s3 engine and s3 table function denotes the size of a file inside the archive, not a size of the archive itself. [#65993](https://github.com/ClickHouse/ClickHouse/pull/65993) ([Daniil Ivanik](https://github.com/divanik)). +* Fix resolving dynamic subcolumns in analyzer, avoid reading the whole column on dynamic subcolumn reading. [#66004](https://github.com/ClickHouse/ClickHouse/pull/66004) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix config merging for from_env with replace overrides. [#66034](https://github.com/ClickHouse/ClickHouse/pull/66034) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible hanging in `GRPCServer` during shutdown. [#66061](https://github.com/ClickHouse/ClickHouse/pull/66061) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` peremeter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixed several cases in function `has` with non-constant `LowCardinality` arguments. [#66088](https://github.com/ClickHouse/ClickHouse/pull/66088) ([Anton Popov](https://github.com/CurtizJ)). +* Fix for `groupArrayIntersect`. It had incorrect behavior in the `merge()` function. Also, fixed behavior in `deserialise()` for numeric and general data. [#66103](https://github.com/ClickHouse/ClickHouse/pull/66103) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the issue when the server failed to parse Avro files with negative block size arrays encoded, which is now allowed by the Avro specification. [#66130](https://github.com/ClickHouse/ClickHouse/pull/66130) ([Serge Klochkov](https://github.com/slvrtrn)). +* Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Fix rare case with missing data in the result of distributed query. [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Fix order of parsing metadata fields in StorageDeltaLake. [#66211](https://github.com/ClickHouse/ClickHouse/pull/66211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error reporting while copying to S3 or AzureBlobStorage. [#66295](https://github.com/ClickHouse/ClickHouse/pull/66295) ([Vitaly Baranov](https://github.com/vitlibar)). +* Prevent watchdog from keeping descriptors of unlinked(rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix the bug that logicalexpressionoptimizerpass lost logical type of constant. [#66344](https://github.com/ClickHouse/ClickHouse/pull/66344) ([pn](https://github.com/chloro-pn)). +* Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too aggressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. [#66402](https://github.com/ClickHouse/ClickHouse/pull/66402) ([vdimir](https://github.com/vdimir)). +* Added missing column materialization for cross join. [#66413](https://github.com/ClickHouse/ClickHouse/pull/66413) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid possible logical error during import from Npy format in case of bad array nesting level, fix testing of other kinds of errors. [#66461](https://github.com/ClickHouse/ClickHouse/pull/66461) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix wrong count() result when there is non-deterministic function in predicate. [#66510](https://github.com/ClickHouse/ClickHouse/pull/66510) ([Duc Canh Le](https://github.com/canhld94)). +* Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading of uninitialized memory when hashing empty tuples. [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix `column_length` is not updated in `ColumnTuple::insertManyFrom`. [#66626](https://github.com/ClickHouse/ClickHouse/pull/66626) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix rare case of stuck merge after drop column. [#66707](https://github.com/ClickHouse/ClickHouse/pull/66707) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assertion `isUniqTypes` when insert select from remote sources. [#66722](https://github.com/ClickHouse/ClickHouse/pull/66722) ([Sema Checherinda](https://github.com/CheSema)). +* Fix logical error in PrometheusRequestHandler. [#66621](https://github.com/ClickHouse/ClickHouse/pull/66621) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `indexHint` function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Fix AST formatting of 'create table b empty as a'. [#64951](https://github.com/ClickHouse/ClickHouse/pull/64951) ([Michael Kolupaev](https://github.com/al13n321)). + +#### Build/Testing/Packaging Improvement +* Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). +* Upgraded `pocketfft` dependency to the recent commit https://github.com/mreineck/pocketfft/commit/f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3. [#66291](https://github.com/ClickHouse/ClickHouse/pull/66291) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Upgraded `azure-sdk-for-cpp` to the recent commit https://github.com/ClickHouse/azure-sdk-for-cpp/commit/ea3e19a7be08519134c643177d56c7484dfec884. [#66292](https://github.com/ClickHouse/ClickHouse/pull/66292) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + ### ClickHouse release 24.6, 2024-07-01 #### Backward Incompatible Change From 73ab9a1197f0d7e71be7c8014621de7226ad4da0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Jul 2024 19:11:22 +0200 Subject: [PATCH 053/123] Revert "Merge pull request #65298 from Algunenano/low_cardinality_wtf" This reverts commit d3a269c61dfa784414245b01ec4d8ccccfeb8b3a, reversing changes made to 023cf118dc77cc316b158a543a5be51ae67b65f9. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +- src/Functions/IFunction.cpp | 101 ++++++------------ src/Planner/Planner.cpp | 7 +- src/Planner/PlannerActionsVisitor.cpp | 61 +++-------- src/Planner/PlannerActionsVisitor.h | 9 +- ..._no_aggregates_and_constant_keys.reference | 4 +- ...nality_group_by_distributed_plan.reference | 55 ---------- ..._cardinality_group_by_distributed_plan.sql | 80 -------------- 8 files changed, 56 insertions(+), 265 deletions(-) delete mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference delete mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index a43e3e62ebc..b1fe2554988 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4124,9 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo auto * column_to_interpolate = interpolate_node_typed.getExpression()->as(); if (!column_to_interpolate) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "INTERPOLATE can work only for identifiers, but {} is found", + throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found", interpolate_node_typed.getExpression()->formatASTForErrorMessage()); auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName(); diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 8b092ba9b6e..31695fc95d5 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -47,85 +47,54 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args) return true; } -/// Replaces single low cardinality column in a function call by its dictionary -/// This can only happen after the arguments have been adapted in IFunctionOverloadResolver::getReturnType -/// as it's only possible if there is one low cardinality column and, optionally, const columns ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { - /// We return the LC indexes so the LC can be reconstructed with the function result + size_t num_rows = input_rows_count; ColumnPtr indexes; - size_t number_low_cardinality_columns = 0; - size_t last_low_cardinality = 0; - size_t number_const_columns = 0; - size_t number_full_columns = 0; - - for (size_t i = 0; i < args.size(); i++) + /// Find first LowCardinality column and replace it to nested dictionary. + for (auto & column : args) { - auto const & arg = args[i]; - if (checkAndGetColumn(arg.column.get())) + if (const auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { - number_low_cardinality_columns++; - last_low_cardinality = i; + /// Single LowCardinality column is supported now. + if (indexes) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single dictionary argument for function."); + + const auto * low_cardinality_type = checkAndGetDataType(column.type.get()); + + if (!low_cardinality_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incompatible type for LowCardinality column: {}", + column.type->getName()); + + if (can_be_executed_on_default_arguments) + { + /// Normal case, when function can be executed on values' default. + column.column = low_cardinality_column->getDictionary().getNestedColumn(); + indexes = low_cardinality_column->getIndexesPtr(); + } + else + { + /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). + /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. + auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); + column.column = dict_encoded.dictionary; + indexes = dict_encoded.indexes; + } + + num_rows = column.column->size(); + column.type = low_cardinality_type->getDictionaryType(); } - else if (checkAndGetColumn(arg.column.get())) - number_const_columns++; - else - number_full_columns++; } - if (!number_low_cardinality_columns && !number_const_columns) - return nullptr; - - if (number_full_columns > 0 || number_low_cardinality_columns > 1) - { - /// This should not be possible but currently there are multiple tests in CI failing because of it - /// TODO: Fix those cases, then enable this exception -#if 0 - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}", - number_low_cardinality_columns, number_full_columns, number_const_columns); -#else - return nullptr; -#endif - } - else if (number_low_cardinality_columns == 1) - { - auto & lc_arg = args[last_low_cardinality]; - - const auto * low_cardinality_type = checkAndGetDataType(lc_arg.type.get()); - if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", lc_arg.type->getName()); - - const auto * low_cardinality_column = checkAndGetColumn(lc_arg.column.get()); - chassert(low_cardinality_column); - - if (can_be_executed_on_default_arguments) - { - /// Normal case, when function can be executed on values' default. - lc_arg.column = low_cardinality_column->getDictionary().getNestedColumn(); - indexes = low_cardinality_column->getIndexesPtr(); - } - else - { - /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). - /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. - auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); - lc_arg.column = dict_encoded.dictionary; - indexes = dict_encoded.indexes; - } - - /// The new column will have a different number of rows, normally less but occasionally it might be more (NULL) - input_rows_count = lc_arg.column->size(); - lc_arg.type = low_cardinality_type->getDictionaryType(); - } - - /// Change size of constants + /// Change size of constants. for (auto & column : args) { if (const auto * column_const = checkAndGetColumn(column.column.get())) { - column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), input_rows_count); + column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), num_rows); column.type = recursiveRemoveLowCardinality(column.type); } } @@ -301,8 +270,6 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithType bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments(); const auto & dictionary_type = res_low_cardinality_type->getDictionaryType(); - /// The arguments should have been adapted in IFunctionOverloadResolver::getReturnType - /// So there is only one low cardinality column (and optionally some const columns) and no full column ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a35ba69d459..5640d84731f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -745,12 +745,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, { auto & interpolate_node_typed = interpolate_node->as(); - PlannerActionsVisitor planner_actions_visitor( - planner_context, - /* use_column_identifier_as_action_node_name_, (default value)*/ true, - /// Prefer the INPUT to CONSTANT nodes (actions must be non constant) - /* always_use_const_column_for_constant_nodes */ false); - + PlannerActionsVisitor planner_actions_visitor(planner_context); auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 6fd37df11c6..a0c7f9b2516 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -487,33 +487,16 @@ public: return node; } - [[nodiscard]] String addConstantIfNecessary( - const std::string & node_name, const ColumnWithTypeAndName & column, bool always_use_const_column_for_constant_nodes) + const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) { - chassert(column.column != nullptr); auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end() && (!always_use_const_column_for_constant_nodes || it->second->column)) - return {node_name}; - if (it != node_name_to_node.end()) - { - /// There is a node with this name, but it doesn't have a column - /// This likely happens because we executed the query until WithMergeableState with a const node in the - /// WHERE clause and, as the results of headers are materialized, the column was removed - /// Let's add a new column and keep this - String dupped_name{node_name + "_dupped"}; - if (node_name_to_node.find(dupped_name) != node_name_to_node.end()) - return dupped_name; - - const auto * node = &actions_dag.addColumn(column); - node_name_to_node[dupped_name] = node; - return dupped_name; - } + return it->second; const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; - return {node_name}; + return node; } template @@ -542,7 +525,7 @@ public: } private: - std::unordered_map node_name_to_node; + std::unordered_map node_name_to_node; ActionsDAG & actions_dag; QueryTreeNodePtr scope_node; }; @@ -550,11 +533,9 @@ private: class PlannerActionsVisitorImpl { public: - PlannerActionsVisitorImpl( - ActionsDAG & actions_dag, + PlannerActionsVisitorImpl(ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_, - bool always_use_const_column_for_constant_nodes_); + bool use_column_identifier_as_action_node_name_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -614,18 +595,14 @@ private: const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; bool use_column_identifier_as_action_node_name; - bool always_use_const_column_for_constant_nodes; }; -PlannerActionsVisitorImpl::PlannerActionsVisitorImpl( - ActionsDAG & actions_dag, +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_, - bool always_use_const_column_for_constant_nodes_) + bool use_column_identifier_as_action_node_name_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) { actions_stack.emplace_back(actions_dag, nullptr); } @@ -748,16 +725,17 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, always_use_const_column_for_constant_nodes); + actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {final_name, Levels(0)}; + return {constant_node_name, Levels(0)}; + } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node) @@ -886,16 +864,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, always_use_const_column_for_constant_nodes); + actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {final_name, Levels(0)}; + return {column.name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) @@ -1032,19 +1010,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } -PlannerActionsVisitor::PlannerActionsVisitor( - const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_, - bool always_use_const_column_for_constant_nodes_) +PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_) : planner_context(planner_context_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { - PlannerActionsVisitorImpl actions_visitor_impl( - actions_dag, planner_context, use_column_identifier_as_action_node_name, always_use_const_column_for_constant_nodes); + PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); return actions_visitor_impl.visit(expression_node); } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 1dbd149bc4b..6bb32047327 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -27,17 +27,11 @@ using PlannerContextPtr = std::shared_ptr; * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. * 2. For function `in` and its variants, already collected sets from planner context are used. - * 3. When building actions that use CONSTANT nodes, by default we ignore pre-existing INPUTs if those don't have - * a column (a const column always has a column). This is for compatibility with previous headers. We disable this - * behaviour when we explicitly want to override CONSTANT nodes with the input (resolving InterpolateNode for example) */ class PlannerActionsVisitor { public: - explicit PlannerActionsVisitor( - const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_ = true, - bool always_use_const_column_for_constant_nodes_ = true); + explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true); /** Add actions necessary to calculate expression node into expression dag. * Necessary actions are not added in actions dag output. @@ -48,7 +42,6 @@ public: private: const PlannerContextPtr planner_context; bool use_column_identifier_as_action_node_name = true; - bool always_use_const_column_for_constant_nodes = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index fc77ed8a241..63b8a9d14fc 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -8,13 +8,13 @@ 40 41 -41 +0 2 42 2 42 43 -43 +0 11 11 diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference deleted file mode 100644 index 1508c24f410..00000000000 --- a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference +++ /dev/null @@ -1,55 +0,0 @@ --- { echoOn } -SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) -FROM system.one -GROUP BY '666'; -6.666.8 -SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) -FROM remote('127.0.0.{1,1}', 'system.one') -GROUP BY '666'; -6.666.8 --- https://github.com/ClickHouse/ClickHouse/issues/63006 -SELECT - 6, - concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, - concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b -FROM system.one -GROUP BY toNullable(6) - WITH ROLLUP -WITH TOTALS; -6 World666666 \N -6 World666666 \N - -6 World666666 \N -SELECT - 6, - concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, - concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b -FROM remote('127.0.0.1') -GROUP BY toNullable(6) - WITH ROLLUP - WITH TOTALS; -6 World666666 \N -6 World666666 \N - -6 World666666 \N --- { echoOn } -SELECT - '%', - tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), - (toDecimal128(99.67, 6), 36, 61, 14) -FROM dist_03174 -WHERE dummy IN (0, '255') -GROUP BY - toNullable(13), - (99.67, 61, toLowCardinality(14)); -% ('%11default10113%AS%id_02%10101010') (99.67,36,61,14) --- { echoOn } -SELECT - 38, - concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) -FROM set_index_not__fuzz_0 -GROUP BY - toNullable(3), - concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) -FORMAT Null -SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql deleted file mode 100644 index d397d30e285..00000000000 --- a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql +++ /dev/null @@ -1,80 +0,0 @@ --- There are various tests that check that group by keys don't propagate into functions replacing const arguments --- by full (empty) columns - -DROP TABLE IF EXISTS dist_03174; -DROP TABLE IF EXISTS set_index_not__fuzz_0; - --- https://github.com/ClickHouse/ClickHouse/issues/63006 - -SET allow_experimental_analyzer=1; - --- { echoOn } -SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) -FROM system.one -GROUP BY '666'; - -SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) -FROM remote('127.0.0.{1,1}', 'system.one') -GROUP BY '666'; - --- https://github.com/ClickHouse/ClickHouse/issues/63006 -SELECT - 6, - concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, - concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b -FROM system.one -GROUP BY toNullable(6) - WITH ROLLUP -WITH TOTALS; - -SELECT - 6, - concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, - concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b -FROM remote('127.0.0.1') -GROUP BY toNullable(6) - WITH ROLLUP - WITH TOTALS; - --- https://github.com/ClickHouse/ClickHouse/issues/64945 --- { echoOff } -CREATE TABLE dist_03174 AS system.one ENGINE = Distributed(test_cluster_two_shards, system, one, dummy); - --- { echoOn } -SELECT - '%', - tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), - (toDecimal128(99.67, 6), 36, 61, 14) -FROM dist_03174 -WHERE dummy IN (0, '255') -GROUP BY - toNullable(13), - (99.67, 61, toLowCardinality(14)); - --- Parallel replicas --- { echoOff } -CREATE TABLE set_index_not__fuzz_0 -( - `name` String, - `status` Enum8('alive' = 0, 'rip' = 1), - INDEX idx_status status TYPE set(2) GRANULARITY 1 -) -ENGINE = MergeTree() -ORDER BY name; - -INSERT INTO set_index_not__fuzz_0 SELECT * FROM generateRandom() LIMIT 10; - --- { echoOn } -SELECT - 38, - concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) -FROM set_index_not__fuzz_0 -GROUP BY - toNullable(3), - concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) -FORMAT Null -SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; - --- { echoOff } -DROP TABLE IF EXISTS dist_03174; -DROP TABLE IF EXISTS set_index_not__fuzz_0; From 104c3be7b691a8e2c7b8c71789d60868aa062e01 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Jul 2024 16:23:51 +0000 Subject: [PATCH 054/123] Functions [h-r]*: Iterate over input_rows_count where appropriate --- src/Functions/FunctionStringToString.h | 6 ++-- src/Functions/LowerUpperImpl.h | 8 +++-- src/Functions/LowerUpperUTF8Impl.h | 7 +++-- src/Functions/StringHelpers.h | 28 +++++++++-------- .../URL/ExtractFirstSignificantSubdomain.h | 4 +-- src/Functions/URL/cutFragment.cpp | 2 +- src/Functions/URL/cutQueryString.cpp | 2 +- .../URL/cutQueryStringAndFragment.cpp | 2 +- .../URL/cutToFirstSignificantSubdomain.cpp | 2 +- .../cutToFirstSignificantSubdomainCustom.cpp | 4 +-- src/Functions/URL/cutWWW.cpp | 2 +- src/Functions/URL/decodeURLComponent.cpp | 15 +++++----- src/Functions/URL/domain.cpp | 3 +- src/Functions/URL/domain.h | 7 +++-- src/Functions/URL/domainWithoutWWW.cpp | 2 +- .../URL/firstSignificantSubdomain.cpp | 2 +- .../URL/firstSignificantSubdomainCustom.cpp | 4 +-- src/Functions/URL/fragment.cpp | 2 +- src/Functions/URL/path.cpp | 2 +- src/Functions/URL/pathFull.cpp | 2 +- src/Functions/URL/port.cpp | 2 +- src/Functions/URL/protocol.cpp | 2 +- src/Functions/URL/queryString.cpp | 2 +- src/Functions/URL/queryStringAndFragment.cpp | 2 +- src/Functions/URL/topLevelDomain.cpp | 2 +- src/Functions/decodeHTMLComponent.cpp | 11 ++++--- src/Functions/decodeXMLComponent.cpp | 10 +++---- src/Functions/encodeXMLComponent.cpp | 10 +++---- src/Functions/idna.cpp | 20 ++++++------- src/Functions/initcap.cpp | 8 +++-- src/Functions/initcapUTF8.cpp | 5 ++-- src/Functions/normalizeQuery.cpp | 14 +++++---- src/Functions/normalizeString.cpp | 10 +++---- src/Functions/normalizedQueryHash.cpp | 14 ++++----- src/Functions/pointInEllipses.cpp | 10 ++----- src/Functions/punycode.cpp | 30 +++++++++---------- src/Functions/reverse.cpp | 6 ++-- src/Functions/reverse.h | 18 ++++++----- src/Functions/reverseUTF8.cpp | 14 ++++----- src/Functions/soundex.cpp | 12 ++++---- src/Functions/toValidUTF8.cpp | 10 +++---- src/Functions/trim.cpp | 10 +++---- 42 files changed, 169 insertions(+), 159 deletions(-) diff --git a/src/Functions/FunctionStringToString.h b/src/Functions/FunctionStringToString.h index 62d7b09f79e..e0e64e47b49 100644 --- a/src/Functions/FunctionStringToString.h +++ b/src/Functions/FunctionStringToString.h @@ -59,19 +59,19 @@ public: bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr column = arguments[0].column; if (const ColumnString * col = checkAndGetColumn(column.get())) { auto col_res = ColumnString::create(); - Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), input_rows_count); return col_res; } else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column.get())) { auto col_res = ColumnFixedString::create(col_fixed->getN()); - Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars()); + Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars(), input_rows_count); return col_res; } else diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index 72b3ce1ca34..d463ef96e16 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -8,17 +8,19 @@ namespace DB template struct LowerUpperImpl { - static void vector(const ColumnString::Chars & data, + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t /*input_rows_count*/) { res_data.resize_exact(data.size()); res_offsets.assign(offsets); array(data.data(), data.data() + data.size(), res_data.data()); } - static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) + static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data, size_t /*input_rows_count*/) { res_data.resize_exact(data.size()); array(data.data(), data.data() + data.size(), res_data.data()); diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index eebba7b9d5f..eedabca5b22 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -90,7 +90,8 @@ struct LowerUpperUTF8Impl const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { if (data.empty()) return; @@ -98,7 +99,7 @@ struct LowerUpperUTF8Impl bool all_ascii = isAllASCII(data.data(), data.size()); if (all_ascii) { - LowerUpperImpl::vector(data, offsets, res_data, res_offsets); + LowerUpperImpl::vector(data, offsets, res_data, res_offsets, input_rows_count); return; } @@ -107,7 +108,7 @@ struct LowerUpperUTF8Impl array(data.data(), data.data() + data.size(), offsets, res_data.data()); } - static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument"); } diff --git a/src/Functions/StringHelpers.h b/src/Functions/StringHelpers.h index 8f3a87d5d0e..24d8fe86c62 100644 --- a/src/Functions/StringHelpers.h +++ b/src/Functions/StringHelpers.h @@ -62,12 +62,13 @@ using Pos = const char *; template struct ExtractSubstringImpl { - static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - size_t size = offsets.size(); - res_offsets.resize(size); - res_data.reserve(size * Extractor::getReserveLengthForElement()); + res_offsets.resize(input_rows_count); + res_data.reserve(input_rows_count * Extractor::getReserveLengthForElement()); size_t prev_offset = 0; size_t res_offset = 0; @@ -76,7 +77,7 @@ struct ExtractSubstringImpl Pos start; size_t length; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { Extractor::execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); @@ -99,7 +100,7 @@ struct ExtractSubstringImpl res_data.assign(start, length); } - static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } @@ -111,12 +112,13 @@ struct ExtractSubstringImpl template struct CutSubstringImpl { - static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.reserve(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; size_t res_offset = 0; @@ -125,7 +127,7 @@ struct CutSubstringImpl Pos start; size_t length; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * current = reinterpret_cast(&data[prev_offset]); Extractor::execute(current, offsets[i] - prev_offset - 1, start, length); @@ -154,7 +156,7 @@ struct CutSubstringImpl res_data.append(start + length, data.data() + data.size()); } - static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } diff --git a/src/Functions/URL/ExtractFirstSignificantSubdomain.h b/src/Functions/URL/ExtractFirstSignificantSubdomain.h index 0d1b1cac8ef..4316fd2fc3a 100644 --- a/src/Functions/URL/ExtractFirstSignificantSubdomain.h +++ b/src/Functions/URL/ExtractFirstSignificantSubdomain.h @@ -1,8 +1,8 @@ #pragma once #include -#include "domain.h" -#include "tldLookup.h" +#include +#include #include /// TLDType namespace DB diff --git a/src/Functions/URL/cutFragment.cpp b/src/Functions/URL/cutFragment.cpp index 3b99edf61a3..b32c4410190 100644 --- a/src/Functions/URL/cutFragment.cpp +++ b/src/Functions/URL/cutFragment.cpp @@ -1,6 +1,6 @@ #include -#include "fragment.h" #include +#include namespace DB { diff --git a/src/Functions/URL/cutQueryString.cpp b/src/Functions/URL/cutQueryString.cpp index 2886adc2e36..d2fa4a004f2 100644 --- a/src/Functions/URL/cutQueryString.cpp +++ b/src/Functions/URL/cutQueryString.cpp @@ -1,6 +1,6 @@ #include -#include "queryString.h" #include +#include namespace DB { diff --git a/src/Functions/URL/cutQueryStringAndFragment.cpp b/src/Functions/URL/cutQueryStringAndFragment.cpp index 4116b352542..ff427beb277 100644 --- a/src/Functions/URL/cutQueryStringAndFragment.cpp +++ b/src/Functions/URL/cutQueryStringAndFragment.cpp @@ -1,6 +1,6 @@ #include -#include "queryStringAndFragment.h" #include +#include namespace DB { diff --git a/src/Functions/URL/cutToFirstSignificantSubdomain.cpp b/src/Functions/URL/cutToFirstSignificantSubdomain.cpp index 6e64b0b6ab8..454a241c54f 100644 --- a/src/Functions/URL/cutToFirstSignificantSubdomain.cpp +++ b/src/Functions/URL/cutToFirstSignificantSubdomain.cpp @@ -1,6 +1,6 @@ #include #include -#include "ExtractFirstSignificantSubdomain.h" +#include namespace DB diff --git a/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp b/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp index 77f40e465a6..7612b6ea4af 100644 --- a/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp +++ b/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp @@ -1,6 +1,6 @@ #include -#include "ExtractFirstSignificantSubdomain.h" -#include "FirstSignificantSubdomainCustomImpl.h" +#include +#include namespace DB { diff --git a/src/Functions/URL/cutWWW.cpp b/src/Functions/URL/cutWWW.cpp index 992d5128440..ab3fae6b094 100644 --- a/src/Functions/URL/cutWWW.cpp +++ b/src/Functions/URL/cutWWW.cpp @@ -1,6 +1,6 @@ #include #include -#include "protocol.h" +#include #include diff --git a/src/Functions/URL/decodeURLComponent.cpp b/src/Functions/URL/decodeURLComponent.cpp index 05e3fbea3fd..bf4aaa6d5e3 100644 --- a/src/Functions/URL/decodeURLComponent.cpp +++ b/src/Functions/URL/decodeURLComponent.cpp @@ -1,7 +1,7 @@ -#include #include #include #include +#include namespace DB @@ -121,8 +121,10 @@ enum URLCodeStrategy template struct CodeURLComponentImpl { - static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { if (code_strategy == encode) { @@ -134,13 +136,12 @@ struct CodeURLComponentImpl res_data.resize(data.size()); } - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * src_data = reinterpret_cast(&data[prev_offset]); size_t src_size = offsets[i] - prev_offset; @@ -165,7 +166,7 @@ struct CodeURLComponentImpl res_data.resize(res_offset); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by URL functions"); } diff --git a/src/Functions/URL/domain.cpp b/src/Functions/URL/domain.cpp index 443a3323075..68724b57a70 100644 --- a/src/Functions/URL/domain.cpp +++ b/src/Functions/URL/domain.cpp @@ -1,5 +1,4 @@ -#include "domain.h" - +#include #include #include diff --git a/src/Functions/URL/domain.h b/src/Functions/URL/domain.h index 936fb9d5f00..328df76b570 100644 --- a/src/Functions/URL/domain.h +++ b/src/Functions/URL/domain.h @@ -1,9 +1,10 @@ #pragma once -#include "protocol.h" -#include -#include #include +#include +#include + +#include namespace DB { diff --git a/src/Functions/URL/domainWithoutWWW.cpp b/src/Functions/URL/domainWithoutWWW.cpp index f6c8b5c84fc..436aad01b82 100644 --- a/src/Functions/URL/domainWithoutWWW.cpp +++ b/src/Functions/URL/domainWithoutWWW.cpp @@ -1,6 +1,6 @@ #include #include -#include "domain.h" +#include namespace DB { diff --git a/src/Functions/URL/firstSignificantSubdomain.cpp b/src/Functions/URL/firstSignificantSubdomain.cpp index b04f6d882ef..e929aefbc27 100644 --- a/src/Functions/URL/firstSignificantSubdomain.cpp +++ b/src/Functions/URL/firstSignificantSubdomain.cpp @@ -1,6 +1,6 @@ #include #include -#include "ExtractFirstSignificantSubdomain.h" +#include namespace DB diff --git a/src/Functions/URL/firstSignificantSubdomainCustom.cpp b/src/Functions/URL/firstSignificantSubdomainCustom.cpp index c07aa2b3ac8..95e5142667b 100644 --- a/src/Functions/URL/firstSignificantSubdomainCustom.cpp +++ b/src/Functions/URL/firstSignificantSubdomainCustom.cpp @@ -1,6 +1,6 @@ #include -#include "ExtractFirstSignificantSubdomain.h" -#include "FirstSignificantSubdomainCustomImpl.h" +#include +#include namespace DB diff --git a/src/Functions/URL/fragment.cpp b/src/Functions/URL/fragment.cpp index 262c1a4c7a6..66da5529d84 100644 --- a/src/Functions/URL/fragment.cpp +++ b/src/Functions/URL/fragment.cpp @@ -1,6 +1,6 @@ #include #include -#include "fragment.h" +#include namespace DB { diff --git a/src/Functions/URL/path.cpp b/src/Functions/URL/path.cpp index 8d609f43191..6de8e4fbf95 100644 --- a/src/Functions/URL/path.cpp +++ b/src/Functions/URL/path.cpp @@ -1,7 +1,7 @@ #include #include #include -#include "path.h" +#include #include diff --git a/src/Functions/URL/pathFull.cpp b/src/Functions/URL/pathFull.cpp index 9aacee21fed..deea617eb29 100644 --- a/src/Functions/URL/pathFull.cpp +++ b/src/Functions/URL/pathFull.cpp @@ -1,7 +1,7 @@ #include #include #include -#include "path.h" +#include #include namespace DB diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index c8f50f10a56..fac46281604 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -5,7 +5,7 @@ #include #include #include -#include "domain.h" +#include namespace DB diff --git a/src/Functions/URL/protocol.cpp b/src/Functions/URL/protocol.cpp index 6bed71207f6..1ac395dc554 100644 --- a/src/Functions/URL/protocol.cpp +++ b/src/Functions/URL/protocol.cpp @@ -1,6 +1,6 @@ #include #include -#include "protocol.h" +#include namespace DB diff --git a/src/Functions/URL/queryString.cpp b/src/Functions/URL/queryString.cpp index 7069ce46b38..f6aaf40fc96 100644 --- a/src/Functions/URL/queryString.cpp +++ b/src/Functions/URL/queryString.cpp @@ -1,6 +1,6 @@ #include #include -#include "queryString.h" +#include namespace DB { diff --git a/src/Functions/URL/queryStringAndFragment.cpp b/src/Functions/URL/queryStringAndFragment.cpp index 367a95acbdc..94f1dfa41e3 100644 --- a/src/Functions/URL/queryStringAndFragment.cpp +++ b/src/Functions/URL/queryStringAndFragment.cpp @@ -1,6 +1,6 @@ #include #include -#include "queryStringAndFragment.h" +#include namespace DB { diff --git a/src/Functions/URL/topLevelDomain.cpp b/src/Functions/URL/topLevelDomain.cpp index 25e9f383f60..b3e88832350 100644 --- a/src/Functions/URL/topLevelDomain.cpp +++ b/src/Functions/URL/topLevelDomain.cpp @@ -1,6 +1,6 @@ #include #include -#include "domain.h" +#include namespace DB { diff --git a/src/Functions/decodeHTMLComponent.cpp b/src/Functions/decodeHTMLComponent.cpp index 00a601b77a6..d36bee534a8 100644 --- a/src/Functions/decodeHTMLComponent.cpp +++ b/src/Functions/decodeHTMLComponent.cpp @@ -28,20 +28,20 @@ namespace const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { /// The size of result is always not more than the size of source. /// Because entities decodes to the shorter byte sequence. /// Example: &#xx... &#xx... will decode to UTF-8 byte sequence not longer than 4 bytes. res_data.resize(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * src_data = reinterpret_cast(&data[prev_offset]); size_t src_size = offsets[i] - prev_offset; @@ -55,7 +55,7 @@ namespace res_data.resize(res_offset); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function decodeHTMLComponent cannot work with FixedString argument"); } @@ -64,7 +64,6 @@ namespace static const int max_legal_unicode_value = 0x10FFFF; static const int max_decimal_length_of_unicode_point = 7; /// 1114111 - static size_t execute(const char * src, size_t src_size, char * dst) { const char * src_pos = src; diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index cbbe46fcb8c..8743aa4759d 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -27,20 +27,20 @@ namespace const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { /// The size of result is always not more than the size of source. /// Because entities decodes to the shorter byte sequence. /// Example: &#xx... &#xx... will decode to UTF-8 byte sequence not longer than 4 bytes. res_data.resize(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * src_data = reinterpret_cast(&data[prev_offset]); size_t src_size = offsets[i] - prev_offset; @@ -54,7 +54,7 @@ namespace res_data.resize(res_offset); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function decodeXMLComponent cannot work with FixedString argument"); } diff --git a/src/Functions/encodeXMLComponent.cpp b/src/Functions/encodeXMLComponent.cpp index 64d85ecaeb8..a22917838b7 100644 --- a/src/Functions/encodeXMLComponent.cpp +++ b/src/Functions/encodeXMLComponent.cpp @@ -25,17 +25,17 @@ namespace const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { /// 6 is the maximum size amplification (the maximum length of encoded entity: ") res_data.resize(data.size() * 6); - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * src_data = reinterpret_cast(&data[prev_offset]); size_t src_size = offsets[i] - prev_offset; @@ -49,7 +49,7 @@ namespace res_data.resize(res_offset); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function encodeXML cannot work with FixedString argument"); } diff --git a/src/Functions/idna.cpp b/src/Functions/idna.cpp index 5a7ae3485ba..cf9e855c912 100644 --- a/src/Functions/idna.cpp +++ b/src/Functions/idna.cpp @@ -44,15 +44,15 @@ struct IdnaEncode const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); size_t prev_offset = 0; std::string ascii; - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { const char * value = reinterpret_cast(&data[prev_offset]); const size_t value_length = offsets[row] - prev_offset - 1; @@ -85,7 +85,7 @@ struct IdnaEncode } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } @@ -99,15 +99,15 @@ struct IdnaDecode const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); size_t prev_offset = 0; std::string unicode; - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { const char * ascii = reinterpret_cast(&data[prev_offset]); const size_t ascii_length = offsets[row] - prev_offset - 1; @@ -124,7 +124,7 @@ struct IdnaDecode } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } diff --git a/src/Functions/initcap.cpp b/src/Functions/initcap.cpp index 4661ce117c0..00414b22344 100644 --- a/src/Functions/initcap.cpp +++ b/src/Functions/initcap.cpp @@ -9,10 +9,12 @@ namespace struct InitcapImpl { - static void vector(const ColumnString::Chars & data, + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t /*input_rows_count*/) { if (data.empty()) return; @@ -21,7 +23,7 @@ struct InitcapImpl array(data.data(), data.data() + data.size(), res_data.data()); } - static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) + static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data, size_t) { res_data.resize(data.size()); array(data.data(), data.data() + data.size(), res_data.data()); diff --git a/src/Functions/initcapUTF8.cpp b/src/Functions/initcapUTF8.cpp index 076dcff6622..282d846094e 100644 --- a/src/Functions/initcapUTF8.cpp +++ b/src/Functions/initcapUTF8.cpp @@ -22,7 +22,8 @@ struct InitcapUTF8Impl const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t /*input_rows_count*/) { if (data.empty()) return; @@ -31,7 +32,7 @@ struct InitcapUTF8Impl array(data.data(), data.data() + data.size(), offsets, res_data.data()); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function initcapUTF8 cannot work with FixedString argument"); } diff --git a/src/Functions/normalizeQuery.cpp b/src/Functions/normalizeQuery.cpp index ad9a8903733..1a78bce7d29 100644 --- a/src/Functions/normalizeQuery.cpp +++ b/src/Functions/normalizeQuery.cpp @@ -19,17 +19,19 @@ template struct Impl { static constexpr auto name = keep_names ? "normalizeQueryKeepNames" : "normalizeQuery"; - static void vector(const ColumnString::Chars & data, + + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); res_data.reserve(data.size()); ColumnString::Offset prev_src_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { ColumnString::Offset curr_src_offset = offsets[i]; @@ -43,7 +45,7 @@ struct Impl } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot apply function normalizeQuery to fixed string."); } diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp index 92411490eaa..c56508ced0e 100644 --- a/src/Functions/normalizeString.cpp +++ b/src/Functions/normalizeString.cpp @@ -84,7 +84,8 @@ struct NormalizeUTF8Impl static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { UErrorCode err = U_ZERO_ERROR; @@ -92,8 +93,7 @@ struct NormalizeUTF8Impl if (U_FAILURE(err)) throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (getNormalizer): {}", u_errorName(err)); - size_t size = offsets.size(); - res_offsets.resize(size); + res_offsets.resize(input_rows_count); res_data.reserve(data.size() * 2); @@ -103,7 +103,7 @@ struct NormalizeUTF8Impl PODArray from_uchars; PODArray to_uchars; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t from_size = offsets[i] - current_from_offset - 1; @@ -157,7 +157,7 @@ struct NormalizeUTF8Impl res_data.resize(current_to_offset); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot apply function normalizeUTF8 to fixed string."); } diff --git a/src/Functions/normalizedQueryHash.cpp b/src/Functions/normalizedQueryHash.cpp index 63218f28af5..3dbe3ff9847 100644 --- a/src/Functions/normalizedQueryHash.cpp +++ b/src/Functions/normalizedQueryHash.cpp @@ -27,13 +27,13 @@ struct Impl static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - PaddedPODArray & res_data) + PaddedPODArray & res_data, + size_t input_rows_count) { - size_t size = offsets.size(); - res_data.resize(size); + res_data.resize(input_rows_count); ColumnString::Offset prev_src_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { ColumnString::Offset curr_src_offset = offsets[i]; res_data[i] = normalizedQueryHash( @@ -77,15 +77,15 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr column = arguments[0].column; if (const ColumnString * col = checkAndGetColumn(column.get())) { auto col_res = ColumnUInt64::create(); typename ColumnUInt64::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::vector(col->getChars(), col->getOffsets(), vec_res); + vec_res.resize(input_rows_count); + Impl::vector(col->getChars(), col->getOffsets(), vec_res, input_rows_count); return col_res; } else diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index 2147428cee3..ac7a8cc4204 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -91,8 +91,6 @@ private: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const auto size = input_rows_count; - /// Prepare array of ellipses. size_t ellipses_count = (arguments.size() - 2) / 4; std::vector ellipses(ellipses_count); @@ -141,13 +139,11 @@ private: auto dst = ColumnVector::create(); auto & dst_data = dst->getData(); - dst_data.resize(size); + dst_data.resize(input_rows_count); size_t start_index = 0; - for (const auto row : collections::range(0, size)) - { + for (size_t row = 0; row < input_rows_count; ++row) dst_data[row] = isPointInEllipses(col_vec_x->getData()[row], col_vec_y->getData()[row], ellipses.data(), ellipses_count, start_index); - } return dst; } @@ -157,7 +153,7 @@ private: const auto * col_const_y = assert_cast (col_y); size_t start_index = 0; UInt8 res = isPointInEllipses(col_const_x->getValue(), col_const_y->getValue(), ellipses.data(), ellipses_count, start_index); - return DataTypeUInt8().createColumnConst(size, res); + return DataTypeUInt8().createColumnConst(input_rows_count, res); } else { diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index 8004e3731b5..ec1fcfd0a70 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -6,11 +6,11 @@ #include #include -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wnewline-eof" -# include -# include -# pragma clang diagnostic pop +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wnewline-eof" +#include +#include +#pragma clang diagnostic pop namespace DB { @@ -38,16 +38,16 @@ struct PunycodeEncode const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); size_t prev_offset = 0; std::u32string value_utf32; std::string value_puny; - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { const char * value = reinterpret_cast(&data[prev_offset]); const size_t value_length = offsets[row] - prev_offset - 1; @@ -72,7 +72,7 @@ struct PunycodeEncode } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } @@ -86,16 +86,16 @@ struct PunycodeDecode const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t rows = offsets.size(); res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); size_t prev_offset = 0; std::u32string value_utf32; std::string value_utf8; - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { const char * value = reinterpret_cast(&data[prev_offset]); const size_t value_length = offsets[row] - prev_offset - 1; @@ -129,7 +129,7 @@ struct PunycodeDecode } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index d23e48b8d42..94b6634ffbd 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -55,19 +55,19 @@ public: bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr column = arguments[0].column; if (const ColumnString * col = checkAndGetColumn(column.get())) { auto col_res = ColumnString::create(); - ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), input_rows_count); return col_res; } else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column.get())) { auto col_res = ColumnFixedString::create(col_fixed->getN()); - ReverseImpl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars()); + ReverseImpl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars(), input_rows_count); return col_res; } else diff --git a/src/Functions/reverse.h b/src/Functions/reverse.h index 5f999af4297..7c18d72769a 100644 --- a/src/Functions/reverse.h +++ b/src/Functions/reverse.h @@ -9,17 +9,18 @@ namespace DB */ struct ReverseImpl { - static void vector(const ColumnString::Chars & data, + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.resize_exact(data.size()); res_offsets.assign(offsets); - size_t size = offsets.size(); ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (size_t j = prev_offset; j < offsets[i] - 1; ++j) res_data[j] = data[offsets[i] + prev_offset - 2 - j]; @@ -28,12 +29,15 @@ struct ReverseImpl } } - static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) + static void vectorFixed( + const ColumnString::Chars & data, + size_t n, + ColumnString::Chars & res_data, + size_t input_rows_count) { res_data.resize_exact(data.size()); - size_t size = data.size() / n; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) for (size_t j = i * n; j < (i + 1) * n; ++j) res_data[j] = data[(i * 2 + 1) * n - j - 1]; } diff --git a/src/Functions/reverseUTF8.cpp b/src/Functions/reverseUTF8.cpp index 1aee349fa8d..ca57d946b19 100644 --- a/src/Functions/reverseUTF8.cpp +++ b/src/Functions/reverseUTF8.cpp @@ -23,25 +23,25 @@ namespace */ struct ReverseUTF8Impl { - static void vector(const ColumnString::Chars & data, + static void vector( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { bool all_ascii = isAllASCII(data.data(), data.size()); if (all_ascii) { - ReverseImpl::vector(data, offsets, res_data, res_offsets); + ReverseImpl::vector(data, offsets, res_data, res_offsets, input_rows_count); return; } res_data.resize(data.size()); res_offsets.assign(offsets); - size_t size = offsets.size(); - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { ColumnString::Offset j = prev_offset; while (j < offsets[i] - 1) @@ -73,7 +73,7 @@ struct ReverseUTF8Impl } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot apply function reverseUTF8 to fixed string."); } diff --git a/src/Functions/soundex.cpp b/src/Functions/soundex.cpp index fcf1523d1a3..e8bd1b664e3 100644 --- a/src/Functions/soundex.cpp +++ b/src/Functions/soundex.cpp @@ -79,14 +79,14 @@ struct SoundexImpl const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t size = offsets.size(); - res_data.resize(size * (length + 1)); - res_offsets.resize(size); + res_data.resize(input_rows_count * (length + 1)); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * value = reinterpret_cast(&data[prev_offset]); const size_t value_length = offsets[i] - prev_offset - 1; @@ -98,7 +98,7 @@ struct SoundexImpl } } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by soundex function"); } diff --git a/src/Functions/toValidUTF8.cpp b/src/Functions/toValidUTF8.cpp index 41d29d9c494..376732256b0 100644 --- a/src/Functions/toValidUTF8.cpp +++ b/src/Functions/toValidUTF8.cpp @@ -128,16 +128,16 @@ struct ToValidUTF8Impl const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - const size_t offsets_size = offsets.size(); /// It can be larger than that, but we believe it is unlikely to happen. res_data.resize(data.size()); - res_offsets.resize(offsets_size); + res_offsets.resize(input_rows_count); size_t prev_offset = 0; WriteBufferFromVector write_buffer(res_data); - for (size_t i = 0; i < offsets_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * haystack_data = reinterpret_cast(&data[prev_offset]); const size_t haystack_size = offsets[i] - prev_offset - 1; @@ -149,7 +149,7 @@ struct ToValidUTF8Impl write_buffer.finalize(); } - [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by toValidUTF8 function"); } diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index 1f0011b8e99..5703e871423 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -43,10 +43,10 @@ public: const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - size_t size = offsets.size(); - res_offsets.resize_exact(size); + res_offsets.resize_exact(input_rows_count); res_data.reserve_exact(data.size()); size_t prev_offset = 0; @@ -55,7 +55,7 @@ public: const UInt8 * start; size_t length; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); @@ -69,7 +69,7 @@ public: } } - static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Functions trimLeft, trimRight and trimBoth cannot work with FixedString argument"); } From 54a503910299c13dac305049a98e8136ab989fbb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 23 Jul 2024 19:25:38 +0200 Subject: [PATCH 055/123] Count log messages for building set and reusing set from cache --- ..._between_multiple_mutations_tasks_long.sql | 30 +++++++++---------- ...e_big_sets_between_mutation_tasks_long.sql | 29 +++++++++--------- 2 files changed, 30 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index 40f7800fee1..6b0677a80ae 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -31,22 +31,22 @@ DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part SETTINGS select_sequential_consistency=1; SYSTEM FLUSH LOGS; - --- Check that in every mutation there were parts where selected rows count then the size of big sets which will mean that sets were shared --- Also check that there was at least one part that read more rows then the size of set which will mean that the -WITH 10000000 AS rows_in_set +-- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' ) +-- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' ) +WITH ( + SELECT uuid + FROM system.tables + WHERE (database = currentDatabase()) AND (name = '02581_trips') + ) AS table_uuid SELECT - mutation_version, - countIf(read_rows >= rows_in_set) >= 1 as has_parts_for_which_set_was_built, - countIf(read_rows <= rows_in_set) >= 1 as has_parts_that_shared_set -FROM -( - SELECT - CAST(splitByChar('_', part_name)[5], 'UInt64') AS mutation_version, - read_rows - FROM system.part_log - WHERE database = currentDatabase() and (event_date >= yesterday()) AND (`table` = '02581_trips') AND (event_type = 'MutatePart') -) + CAST(splitByChar('_', query_id)[5], 'UInt64') AS mutation_version, -- '5521485f-8a40-4aba-87a2-00342c369563::all_3_3_0_6' + sum(message LIKE 'Created Set with % entries%') >= 1 AS has_parts_for_which_set_was_built, + sum(message LIKE 'Got set from cache%') >= 1 AS has_parts_that_shared_set +FROM system.text_log +WHERE + query_id LIKE concat(CAST(table_uuid, 'String'), '::all\\_%') + AND (event_date >= yesterday()) + AND (message LIKE 'Created Set with % entries%' OR message LIKE 'Got set from cache%') GROUP BY mutation_version ORDER BY mutation_version FORMAT TSVWithNames; DROP TABLE 02581_trips; diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql index 603c7cb7db0..091a9c8171d 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql @@ -59,21 +59,22 @@ SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; SYSTEM FLUSH LOGS; --- Check that in every mutation there were parts where selected rows count then the size of big sets which will mean that sets were shared --- Also check that there was at least one part that read more rows then the size of set which will mean that the -WITH 10000000 AS rows_in_set +-- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' ) +-- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' ) +WITH ( + SELECT uuid + FROM system.tables + WHERE (database = currentDatabase()) AND (name = '02581_trips') + ) AS table_uuid SELECT - mutation_version, - countIf(read_rows >= rows_in_set) >= 1 as has_parts_for_which_set_was_built, - countIf(read_rows <= rows_in_set) >= 1 as has_parts_that_shared_set -FROM -( - SELECT - CAST(splitByChar('_', part_name)[5], 'UInt64') AS mutation_version, - read_rows - FROM system.part_log - WHERE database = currentDatabase() and (event_date >= yesterday()) AND (`table` = '02581_trips') AND (event_type = 'MutatePart') -) + CAST(splitByChar('_', query_id)[5], 'UInt64') AS mutation_version, -- '5521485f-8a40-4aba-87a2-00342c369563::all_3_3_0_6' + sum(message LIKE 'Created Set with % entries%') >= 1 AS has_parts_for_which_set_was_built, + sum(message LIKE 'Got set from cache%') >= 1 AS has_parts_that_shared_set +FROM system.text_log +WHERE + query_id LIKE concat(CAST(table_uuid, 'String'), '::all\\_%') + AND (event_date >= yesterday()) + AND (message LIKE 'Created Set with % entries%' OR message LIKE 'Got set from cache%') GROUP BY mutation_version ORDER BY mutation_version FORMAT TSVWithNames; DROP TABLE 02581_trips; From 1cbbbd107712c066bef673e5d4692a6950d9e85e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 23 Jul 2024 17:35:10 +0000 Subject: [PATCH 056/123] Apply libunwind fix --- contrib/libunwind | 2 +- src/Common/QueryProfiler.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/libunwind b/contrib/libunwind index 8f28e64d158..9b1f47ad8a6 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 +Subproject commit 9b1f47ad8a6fcecbeaaead93bd87756ccf658071 diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 746010b5462..a7717a4288a 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -33,7 +33,7 @@ namespace DB namespace { #if defined(OS_LINUX) - thread_local size_t write_trace_iteration = 0; + //thread_local size_t write_trace_iteration = 0; #endif /// Even after timer_delete() the signal can be delivered, /// since it does not do anything with pending signals. @@ -57,7 +57,7 @@ namespace auto saved_errno = errno; /// We must restore previous value of errno in signal handler. -#if defined(OS_LINUX) +#if defined(OS_LINUX) && false //asdqwe if (info) { int overrun_count = info->si_overrun; @@ -92,7 +92,7 @@ namespace constexpr bool sanitizer = false; #endif - asynchronous_stack_unwinding = true; + //asdqwe asynchronous_stack_unwinding = true; if (sanitizer || 0 == sigsetjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1)) { stack_trace.emplace(signal_context); From 032486951316e518aa31e2548e6a501289bb0b6e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 19:07:56 +0000 Subject: [PATCH 057/123] Split dynamic tests and rewrite them from sh to sql to speed up --- .../03036_dynamic_read_subcolumns.reference | 57 -- .../03036_dynamic_read_subcolumns.sh | 62 --- ...ad_subcolumns_compact_merge_tree.reference | 17 + ...mic_read_subcolumns_compact_merge_tree.sql | 40 ++ ...6_dynamic_read_subcolumns_memory.reference | 17 + .../03036_dynamic_read_subcolumns_memory.sql | 40 ++ ..._read_subcolumns_wide_merge_tree.reference | 17 + ...ynamic_read_subcolumns_wide_merge_tree.sql | 40 ++ ...3037_dynamic_merges_1_horizontal.reference | 60 -- .../03037_dynamic_merges_1_horizontal.sh | 52 -- ..._1_horizontal_compact_merge_tree.reference | 28 + ...merges_1_horizontal_compact_merge_tree.sql | 33 ++ ...s_1_horizontal_compact_wide_tree.reference | 28 + ..._merges_1_horizontal_compact_wide_tree.sql | 33 ++ .../03037_dynamic_merges_1_vertical.reference | 60 -- .../03037_dynamic_merges_1_vertical.sh | 51 -- ...es_1_vertical_compact_merge_tree.reference | 28 + ...c_merges_1_vertical_compact_merge_tree.sql | 33 ++ ...erges_1_vertical_wide_merge_tree.reference | 28 + ...amic_merges_1_vertical_wide_merge_tree.sql | 33 ++ .../03037_dynamic_merges_2.reference | 20 - .../0_stateless/03037_dynamic_merges_2.sh | 45 -- ..._2_horizontal_compact_merge_tree.reference | 3 + ...merges_2_horizontal_compact_merge_tree.sql | 14 + ...ges_2_horizontal_wide_merge_tree.reference | 3 + ...ic_merges_2_horizontal_wide_merge_tree.sql | 14 + ...es_2_vertical_compact_merge_tree.reference | 3 + ...c_merges_2_vertical_compact_merge_tree.sql | 14 + ...erges_2_vertical_wide_merge_tree.reference | 3 + ...amic_merges_2_vertical_wide_merge_tree.sql | 14 + ...ested_dynamic_merges_compact_horizontal.sh | 32 -- ...sted_dynamic_merges_compact_horizontal.sql | 29 + ..._nested_dynamic_merges_compact_vertical.sh | 32 -- ...nested_dynamic_merges_compact_vertical.sql | 29 + ...8_nested_dynamic_merges_wide_horizontal.sh | 32 -- ..._nested_dynamic_merges_wide_horizontal.sql | 29 + ...038_nested_dynamic_merges_wide_vertical.sh | 32 -- ...38_nested_dynamic_merges_wide_vertical.sql | 29 + .../03040_dynamic_type_alters_1.reference | 526 ------------------ .../03040_dynamic_type_alters_1.sh | 77 --- ...type_alters_1_compact_merge_tree.reference | 174 ++++++ ...namic_type_alters_1_compact_merge_tree.sql | 53 ++ ...040_dynamic_type_alters_1_memory.reference | 175 ++++++ .../03040_dynamic_type_alters_1_memory.sql | 53 ++ ...ic_type_alters_1_wide_merge_tree.reference | 174 ++++++ ..._dynamic_type_alters_1_wide_merge_tree.sql | 53 ++ .../03040_dynamic_type_alters_2.reference | 182 ------ .../03040_dynamic_type_alters_2.sh | 57 -- ...type_alters_2_compact_merge_tree.reference | 90 +++ ...namic_type_alters_2_compact_merge_tree.sql | 39 ++ ...ic_type_alters_2_wide_merge_tree.reference | 90 +++ ..._dynamic_type_alters_2_wide_merge_tree.sql | 39 ++ 52 files changed, 1539 insertions(+), 1377 deletions(-) delete mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql delete mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.reference delete mode 100755 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql delete mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_vertical.reference delete mode 100755 tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql delete mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2.reference delete mode 100755 tests/queries/0_stateless/03037_dynamic_merges_2.sh create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql delete mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql delete mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql delete mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql delete mode 100755 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql delete mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1.reference delete mode 100755 tests/queries/0_stateless/03040_dynamic_type_alters_1.sh create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.sql create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.sql delete mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_2.reference delete mode 100755 tests/queries/0_stateless/03040_dynamic_type_alters_2.sh create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.sql diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference deleted file mode 100644 index 36984bc8b9b..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.reference +++ /dev/null @@ -1,57 +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 -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 -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.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh deleted file mode 100755 index 65517061b99..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ /dev/null @@ -1,62 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - - -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" -} - -$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;" - -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;" - -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;" diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.reference new file mode 100644 index 00000000000..d75d75896f7 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.reference @@ -0,0 +1,17 @@ +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_compact_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql new file mode 100644 index 00000000000..66fbf006a8c --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql @@ -0,0 +1,40 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +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; +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; +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; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +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; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.reference new file mode 100644 index 00000000000..d75d75896f7 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.reference @@ -0,0 +1,17 @@ +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_memory.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.sql new file mode 100644 index 00000000000..bb03bdef704 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_memory.sql @@ -0,0 +1,40 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=Memory; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +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; +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; +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; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +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; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.reference new file mode 100644 index 00000000000..d75d75896f7 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.reference @@ -0,0 +1,17 @@ +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_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql new file mode 100644 index 00000000000..00aba3a57b6 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql @@ -0,0 +1,40 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +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; +insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +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; +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; +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; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +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; + +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.reference deleted file mode 100644 index 59297e46330..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.reference +++ /dev/null @@ -1,60 +0,0 @@ -MergeTree compact -test -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String -MergeTree wide -test -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh deleted file mode 100755 index 887b2ed94d7..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ /dev/null @@ -1,52 +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 - - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" - -function test() -{ - echo "test" - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, number from numbers(100000)" - $CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(80000)" - $CH_CLIENT -q "insert into test select number, range(number % 10 + 1) from numbers(70000)" - $CH_CLIENT -q "insert into test select number, toDate(number) from numbers(60000)" - $CH_CLIENT -q "insert into test select number, toDateTime(number) from numbers(50000)" - $CH_CLIENT -q "insert into test select number, NULL from numbers(100000)" - - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, map(number, number) from numbers(200000)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, tuple(number, number) from numbers(10000)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" -test -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference new file mode 100644 index 00000000000..d0d777a5a38 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference @@ -0,0 +1,28 @@ +50000 DateTime +60000 Date +70000 Array(UInt16) +80000 String +100000 None +100000 UInt64 +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +200000 Map(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +10000 Tuple(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql new file mode 100644 index 00000000000..b66fe5e2187 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -0,0 +1,33 @@ +-- Tags: long +set allow_experimental_dynamic_type=1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, 'str_' || toString(number) from numbers(80000); +insert into test select number, range(number % 10 + 1) from numbers(70000); +insert into test select number, toDate(number) from numbers(60000); +insert into test select number, toDateTime(number) from numbers(50000); +insert into test select number, NULL from numbers(100000); + +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, map(number, number) from numbers(200000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, tuple(number, number) from numbers(10000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference new file mode 100644 index 00000000000..d0d777a5a38 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference @@ -0,0 +1,28 @@ +50000 DateTime +60000 Date +70000 Array(UInt16) +80000 String +100000 None +100000 UInt64 +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +200000 Map(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +10000 Tuple(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql new file mode 100644 index 00000000000..8a376b6d7d7 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -0,0 +1,33 @@ +-- Tags: long +set allow_experimental_dynamic_type=1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, 'str_' || toString(number) from numbers(80000); +insert into test select number, range(number % 10 + 1) from numbers(70000); +insert into test select number, toDate(number) from numbers(60000); +insert into test select number, toDateTime(number) from numbers(50000); +insert into test select number, NULL from numbers(100000); + +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, map(number, number) from numbers(200000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, tuple(number, number) from numbers(10000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.reference deleted file mode 100644 index 59297e46330..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.reference +++ /dev/null @@ -1,60 +0,0 @@ -MergeTree compact -test -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String -MergeTree wide -test -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh deleted file mode 100755 index 371ae87c2ef..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ /dev/null @@ -1,51 +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 - - - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" -function test() -{ - echo "test" - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, number from numbers(100000)" - $CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(80000)" - $CH_CLIENT -q "insert into test select number, range(number % 10 + 1) from numbers(70000)" - $CH_CLIENT -q "insert into test select number, toDate(number) from numbers(60000)" - $CH_CLIENT -q "insert into test select number, toDateTime(number) from numbers(50000)" - $CH_CLIENT -q "insert into test select number, NULL from numbers(100000)" - - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, map(number, number) from numbers(200000)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, tuple(number, number) from numbers(10000)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference new file mode 100644 index 00000000000..d0d777a5a38 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference @@ -0,0 +1,28 @@ +50000 DateTime +60000 Date +70000 Array(UInt16) +80000 String +100000 None +100000 UInt64 +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +200000 Map(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +10000 Tuple(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql new file mode 100644 index 00000000000..127b56e727c --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -0,0 +1,33 @@ +-- Tags: long +set allow_experimental_dynamic_type=1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, 'str_' || toString(number) from numbers(80000); +insert into test select number, range(number % 10 + 1) from numbers(70000); +insert into test select number, toDate(number) from numbers(60000); +insert into test select number, toDateTime(number) from numbers(50000); +insert into test select number, NULL from numbers(100000); + +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, map(number, number) from numbers(200000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, tuple(number, number) from numbers(10000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference new file mode 100644 index 00000000000..d0d777a5a38 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference @@ -0,0 +1,28 @@ +50000 DateTime +60000 Date +70000 Array(UInt16) +80000 String +100000 None +100000 UInt64 +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +70000 Array(UInt16) +100000 None +100000 UInt64 +190000 String +200000 Map(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +10000 Tuple(UInt64, UInt64) +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +260000 String +100000 None +100000 UInt64 +200000 Map(UInt64, UInt64) +270000 String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql new file mode 100644 index 00000000000..e5c273cb592 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -0,0 +1,33 @@ +-- Tags: long +set allow_experimental_dynamic_type=1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, 'str_' || toString(number) from numbers(80000); +insert into test select number, range(number % 10 + 1) from numbers(70000); +insert into test select number, toDate(number) from numbers(60000); +insert into test select number, toDateTime(number) from numbers(50000); +insert into test select number, NULL from numbers(100000); + +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, map(number, number) from numbers(200000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, tuple(number, number) from numbers(10000); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2.reference b/tests/queries/0_stateless/03037_dynamic_merges_2.reference deleted file mode 100644 index 420b8185b16..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_2.reference +++ /dev/null @@ -1,20 +0,0 @@ -MergeTree compact + horizontal merge -test -1000000 Array(UInt16) -1000000 String -1000000 UInt64 -MergeTree wide + horizontal merge -test -1000000 Array(UInt16) -1000000 String -1000000 UInt64 -MergeTree compact + vertical merge -test -1000000 Array(UInt16) -1000000 String -1000000 UInt64 -MergeTree wide + vertical merge -test -1000000 Array(UInt16) -1000000 String -1000000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2.sh b/tests/queries/0_stateless/03037_dynamic_merges_2.sh deleted file mode 100755 index 40adbdd4262..00000000000 --- a/tests/queries/0_stateless/03037_dynamic_merges_2.sh +++ /dev/null @@ -1,45 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" - - -function test() -{ - echo "test" - $CH_CLIENT -q "system stop merges test" - $CH_CLIENT -q "insert into test select number, number from numbers(1000000)" - $CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000)" - $CH_CLIENT -q "insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000)" - - $CH_CLIENT -nm -q "system start merges test; optimize table test final;" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact + horizontal merge" -$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;" - -echo "MergeTree wide + horizontal merge" -$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;" - -echo "MergeTree compact + vertical merge" -$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, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide + vertical merge" -$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, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference new file mode 100644 index 00000000000..afd392002e5 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference @@ -0,0 +1,3 @@ +1000000 Array(UInt16) +1000000 String +1000000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql new file mode 100644 index 00000000000..6d7a0dd8c18 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -0,0 +1,14 @@ +-- Tags: long + +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; +system stop merges test; +insert into test select number, number from numbers(1000000); +insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); +insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference new file mode 100644 index 00000000000..afd392002e5 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference @@ -0,0 +1,3 @@ +1000000 Array(UInt16) +1000000 String +1000000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql new file mode 100644 index 00000000000..011d54d2360 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -0,0 +1,14 @@ +-- Tags: long + +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; +system stop merges test; +insert into test select number, number from numbers(1000000); +insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); +insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference new file mode 100644 index 00000000000..afd392002e5 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference @@ -0,0 +1,3 @@ +1000000 Array(UInt16) +1000000 String +1000000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql new file mode 100644 index 00000000000..1a74f9e5417 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -0,0 +1,14 @@ +-- Tags: long + +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; +system stop merges test; +insert into test select number, number from numbers(1000000); +insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); +insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference new file mode 100644 index 00000000000..afd392002e5 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference @@ -0,0 +1,3 @@ +1000000 Array(UInt16) +1000000 String +1000000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql new file mode 100644 index 00000000000..cbc834e9660 --- /dev/null +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -0,0 +1,14 @@ +-- Tags: long + +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; +system stop merges test; +insert into test select number, number from numbers(1000000); +insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); +insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +system start merges test; +optimize table test final; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh deleted file mode 100755 index d4b6d1f4b63..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sh +++ /dev/null @@ -1,32 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --enable_named_columns_in_function_tuple=0" - -$CH_CLIENT -q "drop table if exists test;" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" - -$CH_CLIENT -q "system stop merges test" -$CH_CLIENT -q "insert into test select number, number from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql new file mode 100644 index 00000000000..ff1dc5e7ded --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -0,0 +1,29 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test;; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh deleted file mode 100755 index 39671a297cf..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sh +++ /dev/null @@ -1,32 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --enable_named_columns_in_function_tuple=0" - -$CH_CLIENT -q "drop table if exists test;" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" - -$CH_CLIENT -q "system stop merges test" -$CH_CLIENT -q "insert into test select number, number from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql new file mode 100644 index 00000000000..f9b0101cb87 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -0,0 +1,29 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test;; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh deleted file mode 100755 index d58545c0b13..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sh +++ /dev/null @@ -1,32 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --enable_named_columns_in_function_tuple=0" - -$CH_CLIENT -q "drop table if exists test;" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" - -$CH_CLIENT -q "system stop merges test" -$CH_CLIENT -q "insert into test select number, number from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql new file mode 100644 index 00000000000..5f373d41c7d --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -0,0 +1,29 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test;; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh deleted file mode 100755 index 39671a297cf..00000000000 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sh +++ /dev/null @@ -1,32 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --enable_named_columns_in_function_tuple=0" - -$CH_CLIENT -q "drop table if exists test;" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" - -$CH_CLIENT -q "system stop merges test" -$CH_CLIENT -q "insert into test select number, number from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000)" -$CH_CLIENT -q "insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000)" - -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" -$CH_CLIENT -nm -q "system start merges test; optimize table test final;" -$CH_CLIENT -q "select count(), dynamicType(d) || ':' || dynamicType(d.\`Tuple(a Dynamic(max_types=3))\`.a) as type from test group by type order by count(), type" - -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql new file mode 100644 index 00000000000..36bbc76b8cb --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -0,0 +1,29 @@ +-- Tags: long + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test;; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; + +system stop merges test; +insert into test select number, number from numbers(100000); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference deleted file mode 100644 index a9c785d1e48..00000000000 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference +++ /dev/null @@ -1,526 +0,0 @@ -Memory -initial insert -alter add column 1 -3 None -0 0 \N \N \N 0 -1 1 \N \N \N 0 -2 2 \N \N \N 0 -insert after alter add column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -alter modify column 1 -7 None -8 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -4 UInt64 -7 String -8 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 -1 Date -5 UInt64 -8 String -9 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -19 19 \N \N \N \N 0 -20 20 20 \N 20 \N 0 -21 21 str_21 str_21 \N \N 0 -22 22 1970-01-23 \N \N 1970-01-23 0 -alter modify column 3 -1 Date -5 UInt64 -8 String -9 None -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 \N 3 \N 0 -4 4 4 \N 0 \N 4 \N 0 -5 5 5 \N 0 \N 5 \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 \N 12 \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -insert after alter modify column 3 -1 Date -5 UInt64 -8 String -12 None -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 \N 3 \N 0 -4 4 4 \N 0 \N 4 \N 0 -5 5 5 \N 0 \N 5 \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 \N 12 \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -23 \N \N \N 0 \N \N \N 0 -24 24 24 \N 0 \N \N \N 0 -25 str_25 \N str_25 0 \N \N \N 0 -MergeTree compact -initial insert -alter add column 1 -3 None -0 0 \N \N \N 0 -1 1 \N \N \N 0 -2 2 \N \N \N 0 -insert after alter add column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -alter modify column 1 -7 None -8 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 -1 Date -1 UInt64 -9 None -12 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -19 19 \N \N \N \N 0 -20 20 20 \N 20 \N 0 -21 21 str_21 str_21 \N \N 0 -22 22 1970-01-23 \N \N 1970-01-23 0 -alter modify column 3 -1 Date -1 UInt64 -9 None -12 String -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -insert after alter modify column 3 -1 Date -1 UInt64 -12 None -12 String -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -23 \N \N \N 0 \N \N \N 0 -24 24 24 \N 0 \N \N \N 0 -25 str_25 \N str_25 0 \N \N \N 0 -MergeTree wide -initial insert -alter add column 1 -3 None -0 0 \N \N \N 0 -1 1 \N \N \N 0 -2 2 \N \N \N 0 -insert after alter add column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -alter modify column 1 -7 None -8 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 -1 Date -1 UInt64 -9 None -12 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -19 19 \N \N \N \N 0 -20 20 20 \N 20 \N 0 -21 21 str_21 str_21 \N \N 0 -22 22 1970-01-23 \N \N 1970-01-23 0 -alter modify column 3 -1 Date -1 UInt64 -9 None -12 String -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -insert after alter modify column 3 -1 Date -1 UInt64 -12 None -12 String -0 0 0 \N 0 \N \N \N 0 -1 1 1 \N 0 \N \N \N 0 -2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 -6 6 6 \N 0 str_6 \N \N 0 -7 7 7 \N 0 str_7 \N \N 0 -8 8 8 \N 0 str_8 \N \N 0 -9 9 9 \N 0 \N \N \N 0 -10 10 10 \N 0 \N \N \N 0 -11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 -13 13 13 \N 0 str_13 \N \N 0 -14 14 14 \N 0 \N \N \N 0 -15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 -17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 -19 19 19 \N 0 \N \N \N 0 -20 20 20 \N 0 \N 20 \N 0 -21 21 21 \N 0 str_21 \N \N 0 -22 22 22 \N 0 \N \N 1970-01-23 0 -23 \N \N \N 0 \N \N \N 0 -24 24 24 \N 0 \N \N \N 0 -25 str_25 \N str_25 0 \N \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh b/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh deleted file mode 100755 index 1f2a6a31ad7..00000000000 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh +++ /dev/null @@ -1,77 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1" - -function run() -{ - echo "initial insert" - $CH_CLIENT -q "insert into test select number, number from numbers(3)" - - echo "alter add column 1" - $CH_CLIENT -q "alter table test add column d Dynamic(max_types=3) settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert after alter add column 1" - $CH_CLIENT -q "insert into test select number, number, number from numbers(3, 3)" - $CH_CLIENT -q "insert into test select number, number, 'str_' || toString(number) from numbers(6, 3)" - $CH_CLIENT -q "insert into test select number, number, NULL from numbers(9, 3)" - $CH_CLIENT -q "insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "alter modify column 1" - $CH_CLIENT -q "alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert after alter modify column 1" - $CH_CLIENT -q "insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "alter modify column 2" - $CH_CLIENT -q "alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert after alter modify column 2" - $CH_CLIENT -q "insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "alter modify column 3" - $CH_CLIENT -q "alter table test modify column y Dynamic settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, y.UInt64, y.String, y.\`Tuple(a UInt64)\`.a, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert after alter modify column 3" - $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, y.UInt64, y.String, y.\`Tuple(a UInt64)\`.a, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "Memory" -$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=Memory" -run -$CH_CLIENT -q "drop table test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" -run -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -run -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference new file mode 100644 index 00000000000..2ec301b747b --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference @@ -0,0 +1,174 @@ +initial insert +alter add column 1 +3 None +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 +insert after alter add column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +alter modify column 1 +7 None +8 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +insert after alter modify column 1 +8 None +11 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +alter modify column 2 +8 None +11 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +insert after alter modify column 2 +1 Date +1 UInt64 +9 None +12 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 +alter modify column 3 +1 Date +1 UInt64 +9 None +12 String +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +insert after alter modify column 3 +1 Date +1 UInt64 +12 None +12 String +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql new file mode 100644 index 00000000000..4ab700306d4 --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql @@ -0,0 +1,53 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test; +create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000; +select 'initial insert'; +insert into test select number, number from numbers(3); + +select 'alter add column 1'; +alter table test add column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter add column 1'; +insert into test select number, number, number from numbers(3, 3); +insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); +insert into test select number, number, NULL from numbers(9, 3); +insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 1'; +alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 1'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 2'; +alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 2'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 3'; +alter table test modify column y Dynamic settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 3'; +insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference new file mode 100644 index 00000000000..c592528c3cd --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference @@ -0,0 +1,175 @@ +initial insert +alter add column 1 +3 None +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 +insert after alter add column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +alter modify column 1 +7 None +8 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +insert after alter modify column 1 +8 None +11 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +alter modify column 2 +4 UInt64 +7 String +8 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +insert after alter modify column 2 +1 Date +5 UInt64 +8 String +9 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 +alter modify column 3 +1 Date +5 UInt64 +8 String +9 None +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 \N 12 \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +insert after alter modify column 3 +1 Date +5 UInt64 +8 String +12 None +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 \N 12 \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.sql new file mode 100644 index 00000000000..e802fd034ce --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.sql @@ -0,0 +1,53 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test; +create table test (x UInt64, y UInt64) engine=Memory; +select 'initial insert'; +insert into test select number, number from numbers(3); + +select 'alter add column 1'; +alter table test add column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter add column 1'; +insert into test select number, number, number from numbers(3, 3); +insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); +insert into test select number, number, NULL from numbers(9, 3); +insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 1'; +alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 1'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 2'; +alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 2'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 3'; +alter table test modify column y Dynamic settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 3'; +insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference new file mode 100644 index 00000000000..2ec301b747b --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference @@ -0,0 +1,174 @@ +initial insert +alter add column 1 +3 None +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 +insert after alter add column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +alter modify column 1 +7 None +8 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +insert after alter modify column 1 +8 None +11 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +alter modify column 2 +8 None +11 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +insert after alter modify column 2 +1 Date +1 UInt64 +9 None +12 String +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 +alter modify column 3 +1 Date +1 UInt64 +9 None +12 String +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +insert after alter modify column 3 +1 Date +1 UInt64 +12 None +12 String +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.sql new file mode 100644 index 00000000000..55c4f0b5f0c --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.sql @@ -0,0 +1,53 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test; +create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; +select 'initial insert'; +insert into test select number, number from numbers(3); + +select 'alter add column 1'; +alter table test add column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter add column 1'; +insert into test select number, number, number from numbers(3, 3); +insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); +insert into test select number, number, NULL from numbers(9, 3); +insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 1'; +alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 1'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 2'; +alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 2'; +insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter modify column 3'; +alter table test modify column y Dynamic settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter modify column 3'; +insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference deleted file mode 100644 index f7c00bd8c44..00000000000 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference +++ /dev/null @@ -1,182 +0,0 @@ -MergeTree compact -initial insert -alter add column -3 None -0 0 \N \N \N 0 -1 1 \N \N \N 0 -2 2 \N \N \N 0 -insert after alter add column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -alter rename column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -insert nested dynamic -3 Array(Dynamic) -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 [] [] [] -1 1 \N \N \N \N 0 [] [] [] -2 2 \N \N \N \N 0 [] [] [] -3 3 3 \N 3 \N 0 [] [] [] -4 4 4 \N 4 \N 0 [] [] [] -5 5 5 \N 5 \N 0 [] [] [] -6 6 str_6 str_6 \N \N 0 [] [] [] -7 7 str_7 str_7 \N \N 0 [] [] [] -8 8 str_8 str_8 \N \N 0 [] [] [] -9 9 \N \N \N \N 0 [] [] [] -10 10 \N \N \N \N 0 [] [] [] -11 11 \N \N \N \N 0 [] [] [] -12 12 12 \N 12 \N 0 [] [] [] -13 13 str_13 str_13 \N \N 0 [] [] [] -14 14 \N \N \N \N 0 [] [] [] -15 15 [15] \N \N \N 0 [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N 0 [17] [NULL] [NULL] -alter rename column 2 -3 Array(Dynamic) -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 [] [] [] -1 1 \N \N \N \N 0 [] [] [] -2 2 \N \N \N \N 0 [] [] [] -3 3 3 \N 3 \N 0 [] [] [] -4 4 4 \N 4 \N 0 [] [] [] -5 5 5 \N 5 \N 0 [] [] [] -6 6 str_6 str_6 \N \N 0 [] [] [] -7 7 str_7 str_7 \N \N 0 [] [] [] -8 8 str_8 str_8 \N \N 0 [] [] [] -9 9 \N \N \N \N 0 [] [] [] -10 10 \N \N \N \N 0 [] [] [] -11 11 \N \N \N \N 0 [] [] [] -12 12 12 \N 12 \N 0 [] [] [] -13 13 str_13 str_13 \N \N 0 [] [] [] -14 14 \N \N \N \N 0 [] [] [] -15 15 [15] \N \N \N 0 [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N 0 [17] [NULL] [NULL] -MergeTree wide -initial insert -alter add column -3 None -0 0 \N \N \N 0 -1 1 \N \N \N 0 -2 2 \N \N \N 0 -insert after alter add column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -alter rename column 1 -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 \N 3 \N 0 -4 4 4 \N 4 \N 0 -5 5 5 \N 5 \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 \N 12 \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -insert nested dynamic -3 Array(Dynamic) -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 [] [] [] -1 1 \N \N \N \N 0 [] [] [] -2 2 \N \N \N \N 0 [] [] [] -3 3 3 \N 3 \N 0 [] [] [] -4 4 4 \N 4 \N 0 [] [] [] -5 5 5 \N 5 \N 0 [] [] [] -6 6 str_6 str_6 \N \N 0 [] [] [] -7 7 str_7 str_7 \N \N 0 [] [] [] -8 8 str_8 str_8 \N \N 0 [] [] [] -9 9 \N \N \N \N 0 [] [] [] -10 10 \N \N \N \N 0 [] [] [] -11 11 \N \N \N \N 0 [] [] [] -12 12 12 \N 12 \N 0 [] [] [] -13 13 str_13 str_13 \N \N 0 [] [] [] -14 14 \N \N \N \N 0 [] [] [] -15 15 [15] \N \N \N 0 [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N 0 [17] [NULL] [NULL] -alter rename column 2 -3 Array(Dynamic) -4 String -4 UInt64 -7 None -0 0 \N \N \N \N 0 [] [] [] -1 1 \N \N \N \N 0 [] [] [] -2 2 \N \N \N \N 0 [] [] [] -3 3 3 \N 3 \N 0 [] [] [] -4 4 4 \N 4 \N 0 [] [] [] -5 5 5 \N 5 \N 0 [] [] [] -6 6 str_6 str_6 \N \N 0 [] [] [] -7 7 str_7 str_7 \N \N 0 [] [] [] -8 8 str_8 str_8 \N \N 0 [] [] [] -9 9 \N \N \N \N 0 [] [] [] -10 10 \N \N \N \N 0 [] [] [] -11 11 \N \N \N \N 0 [] [] [] -12 12 12 \N 12 \N 0 [] [] [] -13 13 str_13 str_13 \N \N 0 [] [] [] -14 14 \N \N \N \N 0 [] [] [] -15 15 [15] \N \N \N 0 [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N 0 [17] [NULL] [NULL] diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2.sh b/tests/queries/0_stateless/03040_dynamic_type_alters_2.sh deleted file mode 100755 index 6491e64372f..00000000000 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_2.sh +++ /dev/null @@ -1,57 +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 - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1" - -function run() -{ - echo "initial insert" - $CH_CLIENT -q "insert into test select number, number from numbers(3)" - - echo "alter add column" - $CH_CLIENT -q "alter table test add column d Dynamic settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert after alter add column 1" - $CH_CLIENT -q "insert into test select number, number, number from numbers(3, 3)" - $CH_CLIENT -q "insert into test select number, number, 'str_' || toString(number) from numbers(6, 3)" - $CH_CLIENT -q "insert into test select number, number, NULL from numbers(9, 3)" - $CH_CLIENT -q "insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3)" - $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" - $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x" - - echo "alter rename column 1" - $CH_CLIENT -q "alter table test rename column d to d1 settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1)" - $CH_CLIENT -q "select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.\`Tuple(a UInt64)\`.a from test order by x" - - echo "insert nested dynamic" - $CH_CLIENT -q "insert into test select number, number, [number % 2 ? number : 'str_' || toString(number)]::Array(Dynamic) from numbers(15, 3)" - $CH_CLIENT -q "select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1)" - $CH_CLIENT -q "select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.\`Tuple(a UInt64)\`.a, d1.\`Array(Dynamic)\`.UInt64, d1.\`Array(Dynamic)\`.String, d1.\`Array(Dynamic)\`.Date from test order by x" - - echo "alter rename column 2" - $CH_CLIENT -q "alter table test rename column d1 to d2 settings mutations_sync=1" - $CH_CLIENT -q "select count(), dynamicType(d2) from test group by dynamicType(d2) order by count(), dynamicType(d2)" - $CH_CLIENT -q "select x, y, d2, d2.String, d2.UInt64, d2.Date, d2.\`Tuple(a UInt64)\`.a, d2.\`Array(Dynamic)\`.UInt64, d2.\`Array(Dynamic)\`.String, d2.\`Array(Dynamic)\`.Date, from test order by x" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" -run -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -run -$CH_CLIENT -q "drop table test;" - diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.reference new file mode 100644 index 00000000000..a2f2a19805d --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.reference @@ -0,0 +1,90 @@ +initial insert +alter add column +3 None +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 +insert after alter add column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +alter rename column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +insert nested dynamic +3 Array(Dynamic) +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] +alter rename column 2 +3 Array(Dynamic) +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.sql new file mode 100644 index 00000000000..cead110dd7d --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_2_compact_merge_tree.sql @@ -0,0 +1,39 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test; +create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000; + +select 'initial insert'; +insert into test select number, number from numbers(3); + +select 'alter add column'; +alter table test add column d Dynamic settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter add column 1'; +insert into test select number, number, number from numbers(3, 3); +insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); +insert into test select number, number, NULL from numbers(9, 3); +insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter rename column 1'; +alter table test rename column d to d1 settings mutations_sync=1; +select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1); +select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.`Tuple(a UInt64)`.a from test order by x; + +select 'insert nested dynamic'; +insert into test select number, number, [number % 2 ? number : 'str_' || toString(number)]::Array(Dynamic) from numbers(15, 3); +select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1); +select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.`Tuple(a UInt64)`.a, d1.`Array(Dynamic)`.UInt64, d1.`Array(Dynamic)`.String, d1.`Array(Dynamic)`.Date from test order by x; + +select 'alter rename column 2'; +alter table test rename column d1 to d2 settings mutations_sync=1; +select count(), dynamicType(d2) from test group by dynamicType(d2) order by count(), dynamicType(d2); +select x, y, d2, d2.String, d2.UInt64, d2.Date, d2.`Tuple(a UInt64)`.a, d2.`Array(Dynamic)`.UInt64, d2.`Array(Dynamic)`.String, d2.`Array(Dynamic)`.Date, from test order by x; + +drop table test; diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.reference new file mode 100644 index 00000000000..a2f2a19805d --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.reference @@ -0,0 +1,90 @@ +initial insert +alter add column +3 None +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 +insert after alter add column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +alter rename column 1 +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +insert nested dynamic +3 Array(Dynamic) +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] +alter rename column 2 +3 Array(Dynamic) +4 String +4 UInt64 +7 None +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.sql new file mode 100644 index 00000000000..f58599b1d61 --- /dev/null +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_2_wide_merge_tree.sql @@ -0,0 +1,39 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test; +create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; + +select 'initial insert'; +insert into test select number, number from numbers(3); + +select 'alter add column'; +alter table test add column d Dynamic settings mutations_sync=1; +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; + +select 'insert after alter add column 1'; +insert into test select number, number, number from numbers(3, 3); +insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); +insert into test select number, number, NULL from numbers(9, 3); +insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); +select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; + +select 'alter rename column 1'; +alter table test rename column d to d1 settings mutations_sync=1; +select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1); +select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.`Tuple(a UInt64)`.a from test order by x; + +select 'insert nested dynamic'; +insert into test select number, number, [number % 2 ? number : 'str_' || toString(number)]::Array(Dynamic) from numbers(15, 3); +select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1); +select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.`Tuple(a UInt64)`.a, d1.`Array(Dynamic)`.UInt64, d1.`Array(Dynamic)`.String, d1.`Array(Dynamic)`.Date from test order by x; + +select 'alter rename column 2'; +alter table test rename column d1 to d2 settings mutations_sync=1; +select count(), dynamicType(d2) from test group by dynamicType(d2) order by count(), dynamicType(d2); +select x, y, d2, d2.String, d2.UInt64, d2.Date, d2.`Tuple(a UInt64)`.a, d2.`Array(Dynamic)`.UInt64, d2.`Array(Dynamic)`.String, d2.`Array(Dynamic)`.Date, from test order by x; + +drop table test; From 6a3f49ada4d8319edf34fdf91d5eee007df48e26 Mon Sep 17 00:00:00 2001 From: Linh Giang <165205637+linhgiang24@users.noreply.github.com> Date: Tue, 23 Jul 2024 13:23:35 -0600 Subject: [PATCH 058/123] Included "ON CLUSTER cluster_name" syntax for RELOAD DICTIONARIES command Included "ON CLUSTER cluster_name" syntax for RELOAD DICTIONARY and RELOAD DICTIONARIES commands to ensure execution on all replicas. Also to conform with the rest of the document. --- docs/en/sql-reference/statements/system.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index e6d3439d2b9..f4780fd41c1 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -18,6 +18,12 @@ Reloads all dictionaries that have been successfully loaded before. By default, dictionaries are loaded lazily (see [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), so instead of being loaded automatically at startup, they are initialized on first access through dictGet function or SELECT from tables with ENGINE = Dictionary. The `SYSTEM RELOAD DICTIONARIES` query reloads such dictionaries (LOADED). Always returns `Ok.` regardless of the result of the dictionary update. +**Syntax** + +```sql +SYSTEM RELOAD DICTIONARIES [ON CLUSTER cluster_name] +``` + ## RELOAD DICTIONARY Completely reloads a dictionary `dictionary_name`, regardless of the state of the dictionary (LOADED / NOT_LOADED / FAILED). @@ -25,6 +31,8 @@ Always returns `Ok.` regardless of the result of updating the dictionary. The status of the dictionary can be checked by querying the `system.dictionaries` table. ``` sql +SYSTEM RELOAD DICTIONARY [ON CLUSTER cluster_name] dictionary_name + SELECT name, status FROM system.dictionaries; ``` From 190421aafa83bf54c3921ae17c91cbad8c28bd1d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 20:38:54 +0100 Subject: [PATCH 059/123] impl --- .../01508_partition_pruning_long.sh | 30 ----- ... 01508_partition_pruning_long_1.reference} | 119 ------------------ .../01508_partition_pruning_long_1.sh | 79 ++++++++++++ .../01508_partition_pruning_long_2.reference | 119 ++++++++++++++++++ ...ries => 01508_partition_pruning_long_2.sh} | 80 +++++------- 5 files changed, 228 insertions(+), 199 deletions(-) delete mode 100755 tests/queries/0_stateless/01508_partition_pruning_long.sh rename tests/queries/0_stateless/{01508_partition_pruning_long.reference => 01508_partition_pruning_long_1.reference} (50%) create mode 100755 tests/queries/0_stateless/01508_partition_pruning_long_1.sh create mode 100644 tests/queries/0_stateless/01508_partition_pruning_long_2.reference rename tests/queries/0_stateless/{01508_partition_pruning_long.queries => 01508_partition_pruning_long_2.sh} (58%) mode change 100644 => 100755 diff --git a/tests/queries/0_stateless/01508_partition_pruning_long.sh b/tests/queries/0_stateless/01508_partition_pruning_long.sh deleted file mode 100755 index 7b56d8bbf03..00000000000 --- a/tests/queries/0_stateless/01508_partition_pruning_long.sh +++ /dev/null @@ -1,30 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-polymorphic-parts, no-random-settings, no-random-merge-tree-settings, no-debug - -# Description of test result: -# Test the correctness of the partition pruning -# -# Script executes queries from a file 01508_partition_pruning_long.queries (1 line = 1 query) -# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -queries="${CURDIR}/01508_partition_pruning_long.queries" -while IFS= read -r sql -do - [ -z "$sql" ] && continue - if [[ "$sql" == select* ]] ; - then - echo "$sql" - ${CLICKHOUSE_CLIENT} --query "$sql" - CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') - ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" - CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') - echo "" - else - ${CLICKHOUSE_CLIENT} --query "$sql" - fi -done < "$queries" diff --git a/tests/queries/0_stateless/01508_partition_pruning_long.reference b/tests/queries/0_stateless/01508_partition_pruning_long_1.reference similarity index 50% rename from tests/queries/0_stateless/01508_partition_pruning_long.reference rename to tests/queries/0_stateless/01508_partition_pruning_long_1.reference index afdb4257505..3ea4cc4f6ee 100644 --- a/tests/queries/0_stateless/01508_partition_pruning_long.reference +++ b/tests/queries/0_stateless/01508_partition_pruning_long_1.reference @@ -123,122 +123,3 @@ select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 2 2 20000 Selected 2/3 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges ---------- tDD ---------------------------- -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); -1 10000 -Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); -1 10000 -Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; -1 10000 -Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; -3 40000 -Selected 3/4 parts by partition key, 3 parts by primary key, 4/4 marks by primary key, 4 marks to read from 3 ranges - -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; -3 40000 -Selected 3/4 parts by partition key, 3 parts by primary key, 4/4 marks by primary key, 4 marks to read from 3 ranges - ---------- sDD ---------------------------- -select uniqExact(_part), count() from sDD; -6 30000 -Selected 6/6 parts by partition key, 6 parts by primary key, 6/6 marks by primary key, 6 marks to read from 6 ranges - -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; -3 9999 -Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges - -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; -2 9999 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; -0 0 -Selected 0/6 parts by partition key, 0 parts by primary key, 0/0 marks by primary key, 0 marks to read from 0 ranges - -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); -3 11440 -Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges - -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); -2 10000 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from sDD where d >= 1598918400000; -4 20000 -Selected 4/6 parts by partition key, 4 parts by primary key, 4/4 marks by primary key, 4 marks to read from 4 ranges - -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; -3 10001 -Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges - ---------- xMM ---------------------------- -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -2 10000 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; -3 10001 -Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; -2 10000 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; -1 1 -Selected 1/6 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -2 5001 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; -1 5000 -Selected 1/6 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; -2 10000 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where a = 1; -3 15000 -Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges - -select uniqExact(_part), count() from xMM where a = 66; -0 0 -Selected 0/6 parts by partition key, 0 parts by primary key, 0/0 marks by primary key, 0 marks to read from 0 ranges - -select uniqExact(_part), count() from xMM where a <> 66; -6 30000 -Selected 6/6 parts by partition key, 6 parts by primary key, 6/6 marks by primary key, 6 marks to read from 6 ranges - -select uniqExact(_part), count() from xMM where a = 2; -2 10000 -Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where a = 1; -2 15000 -Selected 2/5 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -1 10000 -Selected 1/5 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - -select uniqExact(_part), count() from xMM where a <> 66; -5 30000 -Selected 5/5 parts by partition key, 5 parts by primary key, 5/5 marks by primary key, 5 marks to read from 5 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -2 5001 -Selected 2/5 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges - -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; -1 5000 -Selected 1/5 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges - diff --git a/tests/queries/0_stateless/01508_partition_pruning_long_1.sh b/tests/queries/0_stateless/01508_partition_pruning_long_1.sh new file mode 100755 index 00000000000..512cf8f5265 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_long_1.sh @@ -0,0 +1,79 @@ +#!/usr/bin/env bash +# Tags: long, no-polymorphic-parts, no-random-settings, no-random-merge-tree-settings, no-debug + +# Description of test result: +# Test the correctness of the partition pruning +# +# Script executes queries from a file 01508_partition_pruning_long.queries (1 line = 1 query) +# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +while IFS= read -r sql +do + [ -z "$sql" ] && continue + if [[ "$sql" == select* ]] ; + then + echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') + ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') + echo "" + else + ${CLICKHOUSE_CLIENT} --query "$sql" + fi +done <<< " +DROP TABLE IF EXISTS tMM; + +CREATE TABLE tMM(d DateTime('Asia/Istanbul'), a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES tMM; +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); + +SELECT '--------- tMM ----------------------------'; +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00', 'Asia/Istanbul'); +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00', 'Asia/Istanbul'); +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; + +SYSTEM START MERGES tMM; +OPTIMIZE TABLE tMM FINAL; + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; + +DROP TABLE tMM; +" diff --git a/tests/queries/0_stateless/01508_partition_pruning_long_2.reference b/tests/queries/0_stateless/01508_partition_pruning_long_2.reference new file mode 100644 index 00000000000..bc767f17031 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_long_2.reference @@ -0,0 +1,119 @@ +--------- tDD ---------------------------- +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +1 10000 +Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +1 10000 +Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +1 10000 +Selected 1/4 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +3 40000 +Selected 3/4 parts by partition key, 3 parts by primary key, 4/4 marks by primary key, 4 marks to read from 3 ranges + +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; +3 40000 +Selected 3/4 parts by partition key, 3 parts by primary key, 4/4 marks by primary key, 4 marks to read from 3 ranges + +--------- sDD ---------------------------- +select uniqExact(_part), count() from sDD; +6 30000 +Selected 6/6 parts by partition key, 6 parts by primary key, 6/6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; +3 9999 +Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; +2 9999 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; +0 0 +Selected 0/6 parts by partition key, 0 parts by primary key, 0/0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); +3 11440 +Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); +2 10000 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000; +4 20000 +Selected 4/6 parts by partition key, 4 parts by primary key, 4/4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; +3 10001 +Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges + +--------- xMM ---------------------------- +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +2 10000 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +3 10001 +Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +2 10000 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +1 1 +Selected 1/6 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 +Selected 1/6 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +2 10000 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1; +3 15000 +Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where a = 66; +0 0 +Selected 0/6 parts by partition key, 0 parts by primary key, 0/0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from xMM where a <> 66; +6 30000 +Selected 6/6 parts by partition key, 6 parts by primary key, 6/6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from xMM where a = 2; +2 10000 +Selected 2/6 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1; +2 15000 +Selected 2/5 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +1 10000 +Selected 1/5 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where a <> 66; +5 30000 +Selected 5/5 parts by partition key, 5 parts by primary key, 5/5 marks by primary key, 5 marks to read from 5 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 +Selected 2/5 parts by partition key, 2 parts by primary key, 2/2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 +Selected 1/5 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges + diff --git a/tests/queries/0_stateless/01508_partition_pruning_long.queries b/tests/queries/0_stateless/01508_partition_pruning_long_2.sh old mode 100644 new mode 100755 similarity index 58% rename from tests/queries/0_stateless/01508_partition_pruning_long.queries rename to tests/queries/0_stateless/01508_partition_pruning_long_2.sh index 0d64fc05f0f..3f8a89bdb20 --- a/tests/queries/0_stateless/01508_partition_pruning_long.queries +++ b/tests/queries/0_stateless/01508_partition_pruning_long_2.sh @@ -1,15 +1,35 @@ -DROP TABLE IF EXISTS tMM; +#!/usr/bin/env bash +# Tags: long, no-polymorphic-parts, no-random-settings, no-random-merge-tree-settings, no-debug + +# Description of test result: +# Test the correctness of the partition pruning +# +# Script executes queries from a file 01508_partition_pruning_long.queries (1 line = 1 query) +# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +while IFS= read -r sql +do + [ -z "$sql" ] && continue + if [[ "$sql" == select* ]] ; + then + echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') + ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') + echo "" + else + ${CLICKHOUSE_CLIENT} --query "$sql" + fi +done <<< " DROP TABLE IF EXISTS tDD; DROP TABLE IF EXISTS sDD; DROP TABLE IF EXISTS xMM; -CREATE TABLE tMM(d DateTime('Asia/Istanbul'), a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES tMM; -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00', 'Asia/Istanbul') + number*60, number FROM numbers(5000); CREATE TABLE tDD(d DateTime('Asia/Istanbul'),a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES tDD; @@ -34,44 +54,6 @@ INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00', 'Asia/Istanbul') + numb INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00', 'Asia/Istanbul') + number*60, 1, number FROM numbers(5000); -SELECT '--------- tMM ----------------------------'; -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00', 'Asia/Istanbul'); -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00', 'Asia/Istanbul'); -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; - -SYSTEM START MERGES tMM; -OPTIMIZE TABLE tMM FINAL; - -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; - - SELECT '--------- tDD ----------------------------'; SYSTEM START MERGES tDD; OPTIMIZE TABLE tDD FINAL; @@ -116,9 +98,7 @@ select uniqExact(_part), count() from xMM where a <> 66; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; -DROP TABLE tMM; DROP TABLE tDD; DROP TABLE sDD; DROP TABLE xMM; - - +" From beed31a7e38a6484aec156754106570a34dcb5a9 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 20:07:47 +0000 Subject: [PATCH 060/123] Fix use-of-uninitialized-value in JSONExtract* numeric functions --- src/Functions/FunctionsJSON.cpp | 3 ++- .../03209_functions_json_msan_fuzzer_issue.reference | 1 + .../0_stateless/03209_functions_json_msan_fuzzer_issue.sql | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.reference create mode 100644 tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 47040545677..848856c500f 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -739,7 +739,8 @@ public: { NumberType value; - tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error); + if (!tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error)) + return false; auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); return true; diff --git a/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.reference b/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.reference new file mode 100644 index 00000000000..e02f3666d40 --- /dev/null +++ b/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.reference @@ -0,0 +1 @@ +0 0 0 1.1 diff --git a/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.sql b/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.sql new file mode 100644 index 00000000000..a05b07d5971 --- /dev/null +++ b/tests/queries/0_stateless/03209_functions_json_msan_fuzzer_issue.sql @@ -0,0 +1,2 @@ +WITH '{ "v":1.1}' AS raw SELECT JSONExtract(raw, 'float') AS float32_1, JSONExtract(concat(tuple('1970-01-05', 10, materialize(10), 10, 10, 10, toUInt256(10), 10, toNullable(10), 10, 10), materialize(toUInt256(0)), ', ', 2, 2, toLowCardinality(toLowCardinality(2))), raw, toLowCardinality('v'), 'Float32') AS float32_2, JSONExtractFloat(raw) AS float64_1, JSONExtract(raw, 'v', 'double') AS float64_2; + From 7a6233340e5a141a43e3a1ac7a1d33ea1c4d5a8b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Jul 2024 21:57:01 +0000 Subject: [PATCH 061/123] Hardening of test_backup_restore_new test --- .../test_backup_restore_new/test.py | 46 ++++++++++++++----- 1 file changed, 34 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index d8662fad011..5cbabe35b03 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -4,6 +4,7 @@ import re import random import os.path import sys +import uuid from collections import namedtuple from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry, TSV @@ -538,7 +539,8 @@ def test_backup_not_found_or_already_exists(): def test_file_engine(): - backup_name = f"File('/backups/file/')" + id = uuid.uuid4() + backup_name = f"File('/backups/file/{id}/')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -549,6 +551,7 @@ def test_file_engine(): instance.query(f"RESTORE TABLE test.table FROM {backup_name}") assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query("DROP TABLE test.table") def test_database(): @@ -565,7 +568,8 @@ def test_database(): def test_zip_archive(): - backup_name = f"Disk('backups', 'archive.zip')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.zip')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -578,10 +582,12 @@ def test_zip_archive(): instance.query(f"RESTORE TABLE test.table FROM {backup_name}") assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query("DROP TABLE test.table") def test_zip_archive_with_settings(): - backup_name = f"Disk('backups', 'archive_with_settings.zip')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_with_settings_{id}.zip')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -596,10 +602,12 @@ def test_zip_archive_with_settings(): f"RESTORE TABLE test.table FROM {backup_name} SETTINGS password='qwerty'" ) assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query("DROP TABLE test.table") def test_zip_archive_with_bad_compression_method(): - backup_name = f"Disk('backups', 'archive_with_bad_compression_method.zip')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_with_bad_compression_method_{id}.zip')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -617,7 +625,8 @@ def test_zip_archive_with_bad_compression_method(): def test_tar_archive(): - backup_name = f"Disk('backups', 'archive.tar')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -630,10 +639,12 @@ def test_tar_archive(): instance.query(f"RESTORE TABLE test.table FROM {backup_name}") assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query("DROP TABLE test.table") def test_tar_bz2_archive(): - backup_name = f"Disk('backups', 'archive.tar.bz2')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar.bz2')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -649,7 +660,8 @@ def test_tar_bz2_archive(): def test_tar_gz_archive(): - backup_name = f"Disk('backups', 'archive.tar.gz')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar.gz')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -665,7 +677,8 @@ def test_tar_gz_archive(): def test_tar_lzma_archive(): - backup_name = f"Disk('backups', 'archive.tar.lzma')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar.lzma')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -681,7 +694,8 @@ def test_tar_lzma_archive(): def test_tar_zst_archive(): - backup_name = f"Disk('backups', 'archive.tar.zst')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar.zst')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -697,7 +711,8 @@ def test_tar_zst_archive(): def test_tar_xz_archive(): - backup_name = f"Disk('backups', 'archive.tar.xz')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_{id}.tar.xz')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -713,7 +728,8 @@ def test_tar_xz_archive(): def test_tar_archive_with_password(): - backup_name = f"Disk('backups', 'archive_with_password.tar')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_with_password_{id}.tar')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -731,7 +747,8 @@ def test_tar_archive_with_password(): def test_tar_archive_with_bad_compression_method(): - backup_name = f"Disk('backups', 'archive_with_bad_compression_method.tar')" + id = uuid.uuid4() + backup_name = f"Disk('backups', 'archive_with_bad_compression_method_{id}.tar')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -1220,6 +1237,9 @@ def test_system_users_required_privileges(): assert instance.query("SHOW CREATE ROLE r1") == "CREATE ROLE r1\n" assert instance.query("SHOW GRANTS FOR r1") == "" + instance.query("DROP USER u1") + instance.query("DROP ROLE r1") + def test_system_users_async(): instance.query("CREATE USER u1 IDENTIFIED BY 'qwe123' SETTINGS custom_c = 3") @@ -1412,6 +1432,8 @@ def test_system_functions(): assert instance.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV( [[0, "even"], [1, "odd"], [2, "even"]] ) + instance.query("DROP FUNCTION linear_equation") + instance.query("DROP FUNCTION parity_str") def test_backup_partition(): From d04ca9eea617a406be715fdf5d541e8d650b23d0 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Jul 2024 00:08:32 +0200 Subject: [PATCH 062/123] fix --- tests/integration/runner | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index fc13cb9807a..0667541b196 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -243,7 +243,9 @@ if __name__ == "__main__": "-n", "--parallel", action="store", dest="parallel", help="Parallelism" ) - parser.add_argument("--count", action="store", dest="count", help="Repeat count") + parser.add_argument( + "--count", action="store", type=int, dest="count", help="Repeat count" + ) parser.add_argument( "--no-random", @@ -320,7 +322,9 @@ if __name__ == "__main__": parallel_args += "--dist=loadfile" parallel_args += f" -n {args.parallel}".format() - repeat_args = f" --count {args.count}" if args.count > 0 else "" + repeat_args = ( + f" --count {args.count}" if args.count is not None and args.count > 0 else "" + ) rand_args = "" # if not args.no_random: From d460fa9f3618cdba64315bce579a93329463b7ad Mon Sep 17 00:00:00 2001 From: zoomxi <419486879@qq.com> Date: Wed, 24 Jul 2024 09:57:00 +0800 Subject: [PATCH 063/123] create coordinator after the number of replicas to use for the query is determined --- .../ClusterProxy/executeQuery.cpp | 3 --- src/Processors/QueryPlan/ReadFromRemote.cpp | 6 ++--- src/Processors/QueryPlan/ReadFromRemote.h | 1 - .../ParallelReplicasReadingCoordinator.cpp | 4 ++-- .../ParallelReplicasReadingCoordinator.h | 4 ---- .../test.py | 22 +++---------------- 6 files changed, 8 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 5d56ef09127..59f095f7487 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -517,14 +517,11 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - auto coordinator = std::make_shared( - new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); auto read_from_remote = std::make_unique( query_ast, new_cluster, storage_id, - std::move(coordinator), header, processed_stage, new_context, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 29e12c1e613..cf11052cd59 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -362,7 +362,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, - ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, @@ -375,7 +374,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , cluster(cluster_) , query_ast(query_ast_) , storage_id(storage_id_) - , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) , context(context_) , throttler(throttler_) @@ -429,7 +427,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder { shuffled_pool = shard.pool->getShuffledPools(current_settings); shuffled_pool.resize(max_replicas_to_use); - coordinator->adjustParticipatingReplicasCount(max_replicas_to_use); } else { @@ -439,6 +436,9 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } + coordinator + = std::make_shared(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size); + for (size_t i=0; i < max_replicas_to_use; ++i) { IConnections::ReplicaInfo replica_info diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index eb15269155a..1adb26b2915 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -70,7 +70,6 @@ public: ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, - ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 2ba66256116..f46b4de10b7 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1031,7 +1031,7 @@ void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica if (!pimpl) { unavailable_nodes_registered_before_initialization.push_back(replica_number); - if (unavailable_nodes_registered_before_initialization.size() == participating_replicas_count) + if (unavailable_nodes_registered_before_initialization.size() == replicas_count) throw Exception(ErrorCodes::ALL_CONNECTION_TRIES_FAILED, "Can't connect to any replica chosen for query execution"); } else @@ -1061,7 +1061,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) } ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_) - : replicas_count(replicas_count_), participating_replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) + : replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) { } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index c06ef6ef01a..8b463fda395 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -30,15 +30,11 @@ public: /// needed to report total rows to read void setProgressCallback(ProgressCallback callback); - /// Participating replicas count may be less than replicas count in a shard. - void adjustParticipatingReplicasCount(size_t count) { participating_replicas_count = count; } - private: void initialize(CoordinationMode mode); std::mutex mutex; const size_t replicas_count{0}; - size_t participating_replicas_count{0}; size_t mark_segment_size{0}; std::unique_ptr pimpl; ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index e05f72316d0..04e3a54e581 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -33,7 +33,8 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) -def test_skip_all_replicas(start_cluster, skip_unavailable_shards): +@pytest.mark.parametrize("max_parallel_replicas", [2, 3, 100]) +def test_skip_all_replicas(start_cluster, skip_unavailable_shards, max_parallel_replicas): cluster_name = "test_1_shard_3_unavaliable_replicas" table_name = "tt" create_tables(cluster_name, table_name) @@ -43,25 +44,8 @@ def test_skip_all_replicas(start_cluster, skip_unavailable_shards): f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", settings={ "allow_experimental_parallel_reading_from_replicas": 2, - "max_parallel_replicas": 3, + "max_parallel_replicas": max_parallel_replicas, "cluster_for_parallel_replicas": cluster_name, "skip_unavailable_shards": skip_unavailable_shards, }, ) - -@pytest.mark.parametrize("max_parallel_replicas", [2, 3, 100]) -def test_skip_all_participating_replicas(start_cluster, max_parallel_replicas): - cluster_name = "test_1_shard_3_unavaliable_replicas" - table_name = "tt1" - create_tables(cluster_name, table_name) - - with pytest.raises(QueryRuntimeException): - initiator.query( - f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "max_parallel_replicas": max_parallel_replicas, - "cluster_for_parallel_replicas": cluster_name, - "skip_unavailable_shards": 1, - }, - ) From 385dabaa19a41a703d549925aeb6c9ecc9c0a493 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 07:14:32 +0200 Subject: [PATCH 064/123] Fix CLion, which is one of the worst examples of software ever created --- src/Core/Settings.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3d181e33001..8d3a11dac98 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1,5 +1,8 @@ #pragma once +/// CLion freezes for a minute on every keypress in any file including this. +#if !defined(__CLION_IDE__) + #include #include #include @@ -1348,3 +1351,5 @@ struct FormatFactorySettings : public BaseSettings }; } + +#endif From bc899ead26f593f2e09f6f303151539b5b615560 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 07:45:24 +0200 Subject: [PATCH 065/123] Fix the estimation of total rows for the numbers data source --- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index b070bbe739b..4136e2d58b4 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -596,12 +596,12 @@ Pipe ReadFromSystemNumbersStep::makePipe() numbers_storage.step, step_between_chunks); - if (numbers_storage.limit && i == 0) + if (end && i == 0) { - auto rows_appr = itemCountInRange(numbers_storage.offset, *numbers_storage.limit, numbers_storage.step); - if (limit > 0 && limit < rows_appr) - rows_appr = query_info_limit; - source->addTotalRowsApprox(rows_appr); + UInt64 rows_approx = itemCountInRange(numbers_storage.offset, *end, numbers_storage.step); + if (limit > 0 && limit < rows_approx) + rows_approx = query_info_limit; + source->addTotalRowsApprox(rows_approx); } pipe.addSource(std::move(source)); From 0744a761cd86d43e8746bf71619987b70e34238b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 07:46:23 +0200 Subject: [PATCH 066/123] Add a test --- .../0_stateless/03208_numbers_total_rows_approx.reference | 1 + tests/queries/0_stateless/03208_numbers_total_rows_approx.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/03208_numbers_total_rows_approx.reference create mode 100644 tests/queries/0_stateless/03208_numbers_total_rows_approx.sql diff --git a/tests/queries/0_stateless/03208_numbers_total_rows_approx.reference b/tests/queries/0_stateless/03208_numbers_total_rows_approx.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03208_numbers_total_rows_approx.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03208_numbers_total_rows_approx.sql b/tests/queries/0_stateless/03208_numbers_total_rows_approx.sql new file mode 100644 index 00000000000..7855dfb6207 --- /dev/null +++ b/tests/queries/0_stateless/03208_numbers_total_rows_approx.sql @@ -0,0 +1 @@ +SELECT number FROM numbers(2, 1) WHERE number % 2 = 0 SETTINGS max_rows_to_read = 10; From e156db49225d273abe97eef84612f564a7547be9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 07:56:11 +0200 Subject: [PATCH 067/123] Remove the support for Kerberized HDFS --- .../test_storage_kerberized_hdfs/__init__.py | 0 .../configs/hdfs.xml | 13 - .../hdfs_configs/bootstrap.sh | 280 ------------------ .../kerberos_image_config.sh | 140 --------- .../secrets/krb.conf | 25 -- .../secrets/krb_long.conf | 24 -- .../test_storage_kerberized_hdfs/test.py | 155 ---------- 7 files changed, 637 deletions(-) delete mode 100644 tests/integration/test_storage_kerberized_hdfs/__init__.py delete mode 100644 tests/integration/test_storage_kerberized_hdfs/configs/hdfs.xml delete mode 100755 tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh delete mode 100644 tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh delete mode 100644 tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf delete mode 100644 tests/integration/test_storage_kerberized_hdfs/secrets/krb_long.conf delete mode 100644 tests/integration/test_storage_kerberized_hdfs/test.py diff --git a/tests/integration/test_storage_kerberized_hdfs/__init__.py b/tests/integration/test_storage_kerberized_hdfs/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_kerberized_hdfs/configs/hdfs.xml b/tests/integration/test_storage_kerberized_hdfs/configs/hdfs.xml deleted file mode 100644 index 2e9dc6cc06c..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/configs/hdfs.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - /tmp/keytab/clickhouse.keytab - root@TEST.CLICKHOUSE.TECH - kerberos - - - specuser@TEST.CLICKHOUSE.TECH - - - /tmp/kerb_cache - - diff --git a/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh b/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh deleted file mode 100755 index db6921bc1c8..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh +++ /dev/null @@ -1,280 +0,0 @@ -#!/bin/bash - -: "${HADOOP_PREFIX:=/usr/local/hadoop}" - -cat >> $HADOOP_PREFIX/etc/hadoop/hadoop-env.sh < /usr/local/hadoop/etc/hadoop/core-site.xml - -cat >> /usr/local/hadoop/etc/hadoop/core-site.xml << EOF - - hadoop.security.authentication - kerberos - - - hadoop.security.authorization - true - - - fs.defaultFS - hdfs://kerberizedhdfs1:9010 - - - fs.default.name - hdfs://kerberizedhdfs1:9010 - - - -EOF - - -cat > /usr/local/hadoop/etc/hadoop/hdfs-site.xml << EOF - - - - dfs.replication - 1 - - - - dfs.block.access.token.enable - true - - - - dfs.namenode.keytab.file - /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab - - - dfs.namenode.kerberos.principal - hdfs/_HOST@TEST.CLICKHOUSE.TECH - - - dfs.namenode.kerberos.internal.spnego.principal - HTTP/_HOST@TEST.CLICKHOUSE.TECH - - - - dfs.secondary.namenode.keytab.file - /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab - - - dfs.secondary.namenode.kerberos.principal - hdfs/_HOST@TEST.CLICKHOUSE.TECH - - - dfs.secondary.namenode.kerberos.internal.spnego.principal - HTTP/_HOST@TEST.CLICKHOUSE.TECH - - - - dfs.datanode.data.dir.perm - 700 - - - dfs.datanode.address - 0.0.0.0:1004 - - - dfs.datanode.http.address - 0.0.0.0:1006 - - - - dfs.datanode.ipc.address - 0.0.0.0:0 - - - dfs.namenode.secondary.http-address - 0.0.0.0:0 - - - dfs.namenode.backup.address - 0.0.0.0:0 - - - dfs.namenode.backup.http-address - 0.0.0.0:0 - - - - - - dfs.datanode.keytab.file - /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab - - - dfs.datanode.kerberos.principal - hdfs/_HOST@TEST.CLICKHOUSE.TECH - - - - - dfs.webhdfs.enabled - true - - -dfs.encrypt.data.transfer -false - - - dfs.web.authentication.kerberos.principal - HTTP/_HOST@TEST.CLICKHOUSE.TECH - - - dfs.web.authentication.kerberos.keytab - /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab - - -EOF - - - -# cat > /usr/local/hadoop/etc/hadoop/ssl-server.xml << EOF -# -# -# ssl.server.truststore.location -# /usr/local/hadoop/etc/hadoop/conf/hdfs.jks -# -# -# ssl.server.truststore.password -# masterkey -# -# -# ssl.server.keystore.location -# /usr/local/hadoop/etc/hadoop/conf/hdfs.jks -# -# -# ssl.server.keystore.password -# masterkey -# -# -# ssl.server.keystore.keypassword -# masterkey -# -# -# EOF - -cat > /usr/local/hadoop/etc/hadoop/log4j.properties << EOF -# Set everything to be logged to the console -log4j.rootCategory=DEBUG, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Set the default spark-shell log level to WARN. When running the spark-shell, the -# log level for this class is used to overwrite the root logger's log level, so that -# the user can have different defaults for the shell and regular Spark apps. -log4j.logger.org.apache.spark.repl.Main=INFO - -# Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=DEBUG -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR -log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO -log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO -log4j.logger.org.apache.parquet=ERROR -log4j.logger.parquet=ERROR - -# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support -log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL -log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR - -log4j.logger.org.apache.spark.deploy=DEBUG -log4j.logger.org.apache.spark.executor=DEBUG -log4j.logger.org.apache.spark.scheduler=DEBUG -EOF - -useradd -u 1098 hdfs - -# keytool -genkey -alias kerberized_hdfs1.test.clickhouse.com -keyalg rsa -keysize 1024 -dname "CN=kerberized_hdfs1.test.clickhouse.com" -keypass masterkey -keystore /usr/local/hadoop/etc/hadoop/conf/hdfs.jks -storepass masterkey -keytool -genkey -alias kerberizedhdfs1 -keyalg rsa -keysize 1024 -dname "CN=kerberizedhdfs1" -keypass masterkey -keystore /usr/local/hadoop/etc/hadoop/conf/hdfs.jks -storepass masterkey - -chmod g+r /usr/local/hadoop/etc/hadoop/conf/hdfs.jks - - -service sshd start - -# yum --quiet --assumeyes install krb5-workstation.x86_64 -# yum --quiet --assumeyes install tcpdump - -# cd /tmp -# curl http://archive.apache.org/dist/commons/daemon/source/commons-daemon-1.0.15-src.tar.gz -o commons-daemon-1.0.15-src.tar.gz -# tar xzf commons-daemon-1.0.15-src.tar.gz -# cd commons-daemon-1.0.15-src/src/native/unix -# ./configure && make -# cp ./jsvc /usr/local/hadoop/sbin - - -until kinit -kt /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab hdfs/kerberizedhdfs1@TEST.CLICKHOUSE.TECH; do sleep 2; done -echo "KDC is up and ready to go... starting up" - -$HADOOP_PREFIX/sbin/start-dfs.sh -$HADOOP_PREFIX/sbin/start-yarn.sh -$HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver - -chmod a+r /usr/local/hadoop/etc/hadoop/conf/hdfs.keytab # create dedicated keytab for hdfsuser - -$HADOOP_PREFIX/sbin/start-secure-dns.sh -sleep 3 - -/usr/local/hadoop/bin/hdfs dfsadmin -safemode leave - -/usr/local/hadoop/bin/hdfs dfs -mkdir /user/specuser -/usr/local/hadoop/bin/hdfs dfs -chown specuser /user/specuser -echo "chown_completed" | /usr/local/hadoop/bin/hdfs dfs -appendToFile - /preparations_done_marker - -kdestroy - - - -# adduser --groups hdfs hdfsuser - -# /usr/local/hadoop/sbin/hadoop-daemon.sh --config /usr/local/hadoop/etc/hadoop/ --script /usr/local/hadoop/sbin/hdfs start namenode -# /usr/local/hadoop/sbin/hadoop-daemon.sh --config /usr/local/hadoop/etc/hadoop/ --script /usr/local/hadoop/sbin/hdfs start datanode - - -if [[ $1 == "-d" ]]; then - while true; do sleep 1000; done -fi - -if [[ $1 == "-bash" ]]; then - /bin/bash -fi diff --git a/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh deleted file mode 100644 index 45fb93792e0..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh +++ /dev/null @@ -1,140 +0,0 @@ -#!/bin/bash - - -set -x # trace - -: "${REALM:=TEST.CLICKHOUSE.TECH}" -: "${DOMAIN_REALM:=test.clickhouse.com}" -: "${KERB_MASTER_KEY:=masterkey}" -: "${KERB_ADMIN_USER:=admin}" -: "${KERB_ADMIN_PASS:=admin}" - -create_config() { - : "${KDC_ADDRESS:=$(hostname -f)}" - - cat>/etc/krb5.conf</var/kerberos/krb5kdc/kdc.conf< /var/kerberos/krb5kdc/kadm5.acl -} - -create_keytabs() { - rm /tmp/keytab/*.keytab - - - # kadmin.local -q "addprinc -randkey hdfs/kerberizedhdfs1.${DOMAIN_REALM}@${REALM}" - # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/hdfs.keytab hdfs/kerberizedhdfs1.${DOMAIN_REALM}@${REALM}" - - # kadmin.local -q "addprinc -randkey HTTP/kerberizedhdfs1.${DOMAIN_REALM}@${REALM}" - # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/hdfs.keytab HTTP/kerberizedhdfs1.${DOMAIN_REALM}@${REALM}" - kadmin.local -q "addprinc -randkey hdfs/kerberizedhdfs1@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/hdfs.keytab hdfs/kerberizedhdfs1@${REALM}" - - kadmin.local -q "addprinc -randkey HTTP/kerberizedhdfs1@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/hdfs.keytab HTTP/kerberizedhdfs1@${REALM}" - - kadmin.local -q "addprinc -randkey hdfsuser/node1@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab hdfsuser/node1@${REALM}" - kadmin.local -q "addprinc -randkey hdfsuser@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab hdfsuser@${REALM}" - kadmin.local -q "addprinc -randkey root@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab root@${REALM}" - kadmin.local -q "addprinc -randkey specuser@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab specuser@${REALM}" - - chmod g+r /tmp/keytab/clickhouse.keytab -} - -main() { - - if [ ! -f /kerberos_initialized ]; then - create_config - create_db - create_admin_user - start_kdc - - touch /kerberos_initialized - fi - - if [ ! -f /var/kerberos/krb5kdc/principal ]; then - while true; do sleep 1000; done - else - start_kdc - create_keytabs - tail -F /var/log/kerberos/krb5kdc.log - fi - -} - -[[ "$0" == "${BASH_SOURCE[0]}" ]] && main "$@" diff --git a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf b/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf deleted file mode 100644 index dffdcaebe81..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf +++ /dev/null @@ -1,25 +0,0 @@ -[logging] - default = FILE:/var/log/krb5libs.log - kdc = FILE:/var/log/krb5kdc.log - admin_server = FILE:/var/log/kadmind.log - -[libdefaults] - default_realm = TEST.CLICKHOUSE.TECH - dns_lookup_realm = false - dns_lookup_kdc = false - ticket_lifetime = 5s - forwardable = true - rdns = false - default_tgs_enctypes = des3-hmac-sha1 - default_tkt_enctypes = des3-hmac-sha1 - permitted_enctypes = des3-hmac-sha1 - -[realms] - TEST.CLICKHOUSE.TECH = { - kdc = hdfskerberos - admin_server = hdfskerberos - } - -[domain_realm] - .test.clickhouse.com = TEST.CLICKHOUSE.TECH - test.clickhouse.com = TEST.CLICKHOUSE.TECH diff --git a/tests/integration/test_storage_kerberized_hdfs/secrets/krb_long.conf b/tests/integration/test_storage_kerberized_hdfs/secrets/krb_long.conf deleted file mode 100644 index 43c009d2e98..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/secrets/krb_long.conf +++ /dev/null @@ -1,24 +0,0 @@ -[logging] - default = FILE:/var/log/krb5libs.log - kdc = FILE:/var/log/krb5kdc.log - admin_server = FILE:/var/log/kadmind.log - -[libdefaults] - default_realm = TEST.CLICKHOUSE.TECH - dns_lookup_realm = false - dns_lookup_kdc = false - ticket_lifetime = 15d - forwardable = true - default_tgs_enctypes = des3-hmac-sha1 - default_tkt_enctypes = des3-hmac-sha1 - permitted_enctypes = des3-hmac-sha1 - -[realms] - TEST.CLICKHOUSE.TECH = { - kdc = hdfskerberos - admin_server = hdfskerberos - } - -[domain_realm] - .test.clickhouse.com = TEST.CLICKHOUSE.TECH - test.clickhouse.com = TEST.CLICKHOUSE.TECH diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py deleted file mode 100644 index ddfc1f6483d..00000000000 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ /dev/null @@ -1,155 +0,0 @@ -import time -import pytest - -import os - -from helpers.cluster import ClickHouseCluster, is_arm -import subprocess - -if is_arm(): - pytestmark = pytest.mark.skip - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - with_kerberized_hdfs=True, - user_configs=[], - main_configs=["configs/hdfs.xml"], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - except Exception as ex: - print(ex) - raise ex - finally: - cluster.shutdown() - - -def test_read_table(started_cluster): - hdfs_api = started_cluster.hdfs_api - - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - hdfs_api.write_data("/simple_table_function", data) - - api_read = hdfs_api.read_data("/simple_table_function") - assert api_read == data - - select_read = node1.query( - "select * from hdfs('hdfs://kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')" - ) - assert select_read == data - - -def test_read_write_storage(started_cluster): - hdfs_api = started_cluster.hdfs_api - - node1.query( - "create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')" - ) - node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") - - api_read = hdfs_api.read_data("/simple_storage1") - assert api_read == "1\tMark\t72.53\n" - - select_read = node1.query("select * from SimpleHDFSStorage2") - assert select_read == "1\tMark\t72.53\n" - - -def test_write_storage_not_expired(started_cluster): - hdfs_api = started_cluster.hdfs_api - - node1.query( - "create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')" - ) - - time.sleep(15) # wait for ticket expiration - node1.query("insert into SimpleHDFSStorageNotExpired values (1, 'Mark', 72.53)") - - api_read = hdfs_api.read_data("/simple_storage_not_expired") - assert api_read == "1\tMark\t72.53\n" - - select_read = node1.query("select * from SimpleHDFSStorageNotExpired") - assert select_read == "1\tMark\t72.53\n" - - -def test_two_users(started_cluster): - hdfs_api = started_cluster.hdfs_api - - node1.query( - "create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')" - ) - node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") - - node1.query( - "create table HDFSStorTwo (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://suser@kerberizedhdfs1:9010/user/specuser/storage_user_two', 'TSV')" - ) - node1.query("insert into HDFSStorTwo values (1, 'Ideal', 74.00)") - - select_read_1 = node1.query( - "select * from hdfs('hdfs://kerberizedhdfs1:9010/user/specuser/storage_user_two', 'TSV', 'id UInt64, text String, number Float64')" - ) - - select_read_2 = node1.query( - "select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')" - ) - - -def test_read_table_expired(started_cluster): - hdfs_api = started_cluster.hdfs_api - - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - hdfs_api.write_data("/simple_table_function_relogin", data) - - started_cluster.pause_container("hdfskerberos") - time.sleep(15) - - try: - select_read = node1.query( - "select * from hdfs('hdfs://reloginuser&kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')" - ) - assert False, "Exception have to be thrown" - except Exception as ex: - assert "DB::Exception: KerberosInit failure:" in str(ex) - - started_cluster.unpause_container("hdfskerberos") - - -def test_prohibited(started_cluster): - node1.query( - "create table HDFSStorTwoProhibited (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://suser@kerberizedhdfs1:9010/storage_user_two_prohibited', 'TSV')" - ) - try: - node1.query("insert into HDFSStorTwoProhibited values (1, 'SomeOne', 74.00)") - assert False, "Exception have to be thrown" - except Exception as ex: - assert ( - "Unable to open HDFS file: /storage_user_two_prohibited (hdfs://suser@kerberizedhdfs1:9010/storage_user_two_prohibited) error: Permission denied: user=specuser, access=WRITE" - in str(ex) - ) - - -def test_cache_path(started_cluster): - node1.query( - "create table HDFSStorCachePath (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://dedicatedcachepath@kerberizedhdfs1:9010/storage_dedicated_cache_path', 'TSV')" - ) - try: - node1.query("insert into HDFSStorCachePath values (1, 'FatMark', 92.53)") - assert False, "Exception have to be thrown" - except Exception as ex: - assert ( - "DB::Exception: hadoop.security.kerberos.ticket.cache.path cannot be set per user" - in str(ex) - ) - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() From 3a69e1555530f59cc83cb274c7dfa1dcaab14a2d Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 24 Jul 2024 09:32:51 +0200 Subject: [PATCH 068/123] CI: Fix for workflow results parsing --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 2 +- .github/workflows/merge_queue.yml | 2 +- .github/workflows/nightly.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release_branches.yml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c602a46d23c..322946ac77b 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -269,7 +269,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7c319da6045..acd7511d520 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -135,7 +135,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index 4b186241a0e..64083668719 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -108,7 +108,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 84db3338065..ea9c125db70 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -54,7 +54,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c7d7b28af38..63b2bd87dc9 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -168,7 +168,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bca9ff33cd0..b79208b03a6 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -489,7 +489,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF From f39c18d4d0e97771d919c2f4d511b8f8f87dae24 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 10:03:36 +0200 Subject: [PATCH 069/123] Fix flaky 01454_storagememory_data_race_challenge --- .../01454_storagememory_data_race_challenge.sh | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh index fa9238041b1..ec9c5134059 100755 --- a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh +++ b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh @@ -11,12 +11,17 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mem" $CLICKHOUSE_CLIENT -q "CREATE TABLE mem (x UInt64) engine = Memory" function f { + local TIMELIMIT=$((SECONDS+$1)) for _ in $(seq 1 300); do $CLICKHOUSE_CLIENT -q "SELECT count() FROM (SELECT * FROM mem SETTINGS max_threads=2) FORMAT Null;" + if [ $SECONDS -ge "$TIMELIMIT" ]; then + break + fi done } function g { + local TIMELIMIT=$((SECONDS+$1)) for _ in $(seq 1 100); do $CLICKHOUSE_CLIENT -n -q " INSERT INTO mem SELECT number FROM numbers(1000000); @@ -30,14 +35,18 @@ function g { INSERT INTO mem VALUES (1); TRUNCATE TABLE mem; " + if [ $SECONDS -ge "$TIMELIMIT" ]; then + break + fi done } export -f f; export -f g; -timeout 20 bash -c f > /dev/null & -timeout 20 bash -c g > /dev/null & +TIMEOUT=20 +f $TIMEOUT & +g $TIMEOUT & wait $CLICKHOUSE_CLIENT -q "DROP TABLE mem" From 42fe58f0466cf5a74f88c1a48bee9e96540f01e5 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 24 Jul 2024 08:33:00 +0000 Subject: [PATCH 070/123] bugfix AttachedTable counting not symmetry, and adding some test logs on attaching and detaching tables --- src/Databases/DatabasesCommon.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 6ccaf811764..851ba8aef35 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -289,8 +289,12 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n tables.erase(it); table_storage->is_detached = true; - if (!table_storage->isSystemStorage() && database_name != DatabaseCatalog::SYSTEM_DATABASE) + if (!table_storage->isSystemStorage() + && database_name != DatabaseCatalog::SYSTEM_DATABASE + && database_name != DatabaseCatalog::TEMPORARY_DATABASE) { + LOG_TEST(log, "Counting detached table {} to database {}", table_name, database_name); CurrentMetrics::sub(getAttachedCounterForStorage(table_storage)); + } auto table_id = table_storage->getStorageID(); if (table_id.hasUUID()) @@ -334,8 +338,12 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; - if (!table->isSystemStorage() && table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) + if (!table->isSystemStorage() + && database_name != DatabaseCatalog::SYSTEM_DATABASE + && database_name != DatabaseCatalog::TEMPORARY_DATABASE) { + LOG_TEST(log, "Counting attached table {} to database {}", table_name, database_name); CurrentMetrics::add(getAttachedCounterForStorage(table)); + } } void DatabaseWithOwnTablesBase::shutdown() From a43c4f4f7ff5e0f0f63c870a01e6c1c28ee6f267 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 24 Jul 2024 10:54:16 +0200 Subject: [PATCH 071/123] CI: Automerge when required and non-required checks completed --- .github/workflows/pull_request.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 63b2bd87dc9..34bf51871d2 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -152,8 +152,9 @@ jobs: CheckReadyForMerge: if: ${{ !cancelled() }} - # Test_2 or Test_3 must not have jobs required for Mergeable check - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1] + # Test_2 or Test_3 do not have the jobs required for Mergeable check, + # however, set them as "needs" to get all checks results before the automatic merge occurs. + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code From e46d400f7beba9479dfcd0d63f025268081f625c Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 24 Jul 2024 09:03:05 +0000 Subject: [PATCH 072/123] fix style --- src/Databases/DatabasesCommon.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 851ba8aef35..d2926c64f29 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -291,7 +291,8 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n if (!table_storage->isSystemStorage() && database_name != DatabaseCatalog::SYSTEM_DATABASE - && database_name != DatabaseCatalog::TEMPORARY_DATABASE) { + && database_name != DatabaseCatalog::TEMPORARY_DATABASE) + { LOG_TEST(log, "Counting detached table {} to database {}", table_name, database_name); CurrentMetrics::sub(getAttachedCounterForStorage(table_storage)); } @@ -340,7 +341,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c if (!table->isSystemStorage() && database_name != DatabaseCatalog::SYSTEM_DATABASE - && database_name != DatabaseCatalog::TEMPORARY_DATABASE) { + && database_name != DatabaseCatalog::TEMPORARY_DATABASE) + { LOG_TEST(log, "Counting attached table {} to database {}", table_name, database_name); CurrentMetrics::add(getAttachedCounterForStorage(table)); } From 0f9ee5c37d68f0877c0bc982c6bb59bf4803f98e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 11:25:08 +0200 Subject: [PATCH 073/123] Fix test "very_long_arrays" --- tests/queries/0_stateless/00186_very_long_arrays.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00186_very_long_arrays.sh b/tests/queries/0_stateless/00186_very_long_arrays.sh index 739b17ccc99..086303279fa 100755 --- a/tests/queries/0_stateless/00186_very_long_arrays.sh +++ b/tests/queries/0_stateless/00186_very_long_arrays.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-tsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 48b30081265f85502b83c7bc5cc4beb9bfeaa3a1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 12:04:28 +0200 Subject: [PATCH 074/123] Handle better static destructors --- src/Client/ClientBase.cpp | 13 +++++++--- src/Common/FailPoint.cpp | 1 + src/Common/PipeFDs.cpp | 13 ++++++++-- src/Common/PipeFDs.h | 3 --- src/Common/SignalHandlers.cpp | 9 ++++++- src/Common/StackTrace.cpp | 23 ++++++++++++++++- src/Daemon/BaseDaemon.cpp | 17 ++++++++++--- src/Loggers/OwnSplitChannel.cpp | 10 ++++++++ src/Loggers/OwnSplitChannel.h | 2 ++ .../__init__.py | 0 .../test.py | 25 +++++++++++++++++++ 11 files changed, 102 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_shutdown_static_destructor_failure/__init__.py create mode 100644 tests/integration/test_shutdown_static_destructor_failure/test.py diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7af199131b6..7867cf32d24 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -309,9 +309,16 @@ public: ClientBase::~ClientBase() { - writeSignalIDtoSignalPipe(SignalListener::StopThread); - signal_listener_thread.join(); - HandledSignals::instance().reset(); + try + { + writeSignalIDtoSignalPipe(SignalListener::StopThread); + signal_listener_thread.join(); + HandledSignals::instance().reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } ClientBase::ClientBase( diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 7b8b5036af0..f5ec8cf0356 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -59,6 +59,7 @@ static struct InitFiu ONCE(execute_query_calling_empty_set_result_func_on_exception) \ ONCE(receive_timeout_on_table_status_response) \ REGULAR(keepermap_fail_drop_data) \ + REGULAR(lazy_pipe_fds_fail_close) \ namespace FailPoints diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index ceadbb2f983..50eeda1bbe2 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -1,19 +1,23 @@ #include #include #include +#include #include #include #include #include -#include #include - namespace DB { +namespace FailPoints +{ + extern const char lazy_pipe_fds_fail_close[]; +} + namespace ErrorCodes { extern const int CANNOT_PIPE; @@ -42,6 +46,11 @@ void LazyPipeFDs::open() void LazyPipeFDs::close() { + fiu_do_on(FailPoints::lazy_pipe_fds_fail_close, + { + throw Exception(ErrorCodes::CANNOT_PIPE, "Manually triggered exception on close"); + }); + for (int & fd : fds_rw) { if (fd < 0) diff --git a/src/Common/PipeFDs.h b/src/Common/PipeFDs.h index 20bd847c077..b651176ee26 100644 --- a/src/Common/PipeFDs.h +++ b/src/Common/PipeFDs.h @@ -1,8 +1,5 @@ #pragma once -#include - - namespace DB { diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index a4b7784df5c..52c83d80121 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -605,7 +605,14 @@ void HandledSignals::reset() HandledSignals::~HandledSignals() { - reset(); + try + { + reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } }; HandledSignals & HandledSignals::instance() diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 34f6f0b7535..b2ece02e2f0 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -4,6 +4,7 @@ #include #include +#include "Common/PipeFDs.h" #include #include #include @@ -489,11 +490,24 @@ struct CacheEntry using CacheEntryPtr = std::shared_ptr; -using StackTraceCache = std::map>; +static constinit std::atomic can_use_cache = false; + +using StackTraceCacheBase = std::map>; + +struct StackTraceCache : public StackTraceCacheBase +{ + using StackTraceCacheBase::StackTraceCacheBase; + + ~StackTraceCache() + { + can_use_cache = false; + } +}; static StackTraceCache & cacheInstance() { static StackTraceCache cache; + can_use_cache = true; return cache; } @@ -503,6 +517,13 @@ String toStringCached(const StackTrace::FramePointers & pointers, size_t offset, { const StackTraceRefTriple key{pointers, offset, size}; + if (!can_use_cache) + { + DB::WriteBufferFromOwnString out; + toStringEveryLineImpl(false, key, [&](std::string_view str) { out << str << '\n'; }); + return out.str(); + } + /// Calculation of stack trace text is extremely slow. /// We use cache because otherwise the server could be overloaded by trash queries. /// Note that this cache can grow unconditionally, but practically it should be small. diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index d77878415db..366aad00376 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -146,10 +146,19 @@ BaseDaemon::BaseDaemon() = default; BaseDaemon::~BaseDaemon() { - writeSignalIDtoSignalPipe(SignalListener::StopThread); - signal_listener_thread.join(); - HandledSignals::instance().reset(); - SentryWriter::resetInstance(); + try + { + writeSignalIDtoSignalPipe(SignalListener::StopThread); + signal_listener_thread.join(); + HandledSignals::instance().reset(); + SentryWriter::resetInstance(); + } + catch (...) + { + tryLogCurrentException(&logger()); + } + + OwnSplitChannel::disableLogging(); } diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index dc51a13e01f..c0e8514c62a 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -16,8 +16,18 @@ namespace DB { +static constinit std::atomic allow_logging{true}; + +void OwnSplitChannel::disableLogging() +{ + allow_logging = false; +} + void OwnSplitChannel::log(const Poco::Message & msg) { + if (!allow_logging) + return; + #ifndef WITHOUT_TEXT_LOG auto logs_queue = CurrentThread::getInternalTextLogsQueue(); diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 88bb6b9ce76..9872a4fb558 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -39,6 +39,8 @@ public: void setLevel(const std::string & name, int level); + static void disableLogging(); + private: void logSplit(const Poco::Message & msg); void tryLogSplit(const Poco::Message & msg); diff --git a/tests/integration/test_shutdown_static_destructor_failure/__init__.py b/tests/integration/test_shutdown_static_destructor_failure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_shutdown_static_destructor_failure/test.py b/tests/integration/test_shutdown_static_destructor_failure/test.py new file mode 100644 index 00000000000..8f1d4423238 --- /dev/null +++ b/tests/integration/test_shutdown_static_destructor_failure/test.py @@ -0,0 +1,25 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=[], + stay_alive=True +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_shutdown(): + node.query("SYSTEM ENABLE FAILPOINT lazy_pipe_fds_fail_close") + node.stop_clickhouse() From e7b5c0ea19a64157f7bc743f81cf148cf3fdb4eb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Jul 2024 10:14:25 +0000 Subject: [PATCH 075/123] Automatic style fix --- .../test_shutdown_static_destructor_failure/test.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/integration/test_shutdown_static_destructor_failure/test.py b/tests/integration/test_shutdown_static_destructor_failure/test.py index 8f1d4423238..b1d925cc432 100644 --- a/tests/integration/test_shutdown_static_destructor_failure/test.py +++ b/tests/integration/test_shutdown_static_destructor_failure/test.py @@ -4,11 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node", - main_configs=[], - stay_alive=True -) +node = cluster.add_instance("node", main_configs=[], stay_alive=True) @pytest.fixture(scope="module", autouse=True) From 8c19d502f861ca844501fa505481a1b64684a8ec Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 24 Jul 2024 10:44:06 +0000 Subject: [PATCH 076/123] Add decorator and retries for azurite --- tests/integration/helpers/cluster.py | 22 ++++++------ tests/integration/helpers/retry_decorator.py | 36 ++++++++++++++++++++ 2 files changed, 48 insertions(+), 10 deletions(-) create mode 100644 tests/integration/helpers/retry_decorator.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 548b58a17e8..3914192f9af 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -52,6 +52,7 @@ from helpers.client import QueryRuntimeException import docker from .client import Client +from .retry_decorator import retry from .config_cluster import * @@ -2690,15 +2691,12 @@ class ClickHouseCluster: images_pull_cmd = self.base_cmd + ["pull"] # sometimes dockerhub/proxy can be flaky - for i in range(5): - try: - run_and_check(images_pull_cmd) - break - except Exception as ex: - if i == 4: - raise ex - logging.info("Got exception pulling images: %s", ex) - time.sleep(i * 3) + + retry( + log_function=lambda exception: logging.info( + "Got exception pulling images: %s", exception + ), + )(run_and_check)(images_pull_cmd) if self.with_zookeeper_secure and self.base_zookeeper_cmd: logging.debug("Setup ZooKeeper Secure") @@ -2971,7 +2969,11 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - run_and_check(azurite_start_cmd) + retry( + log_function=lambda exception: logginnfo( + f"Azurite initialization failed with error: {exception}" + ), + )(run_and_check)(azurite_start_cmd) self.up_called = True logging.info("Trying to connect to Azurite") self.wait_azurite_to_start() diff --git a/tests/integration/helpers/retry_decorator.py b/tests/integration/helpers/retry_decorator.py new file mode 100644 index 00000000000..aaa040464c2 --- /dev/null +++ b/tests/integration/helpers/retry_decorator.py @@ -0,0 +1,36 @@ +import time +import random +from typing import Type, List + + +def retry( + retries: int = 5, + delay: float = 1, + backoff: float = 1.5, + jitter: float = 2, + log_function=lambda *args, **kwargs: None, + retriable_expections_list: List[Type[BaseException]] = [Exception], +): + def inner(func): + def wrapper(*args, **kwargs): + current_delay = delay + for retry in range(retries): + try: + func(*args, **kwargs) + break + except Exception as e: + should_retry = False + for retriable_exception in retriable_expections_list: + if isinstance(e, retriable_exception): + should_retry = True + break + if not should_retry or (retry == retries - 1): + raise e + log_function(retry=retry, exception=e) + sleep_time = current_delay + random.uniform(0, jitter) + time.sleep(sleep_time) + current_delay *= backoff + + return wrapper + + return inner From 1b95a68dc08266c50b2dafb5c852b84431723112 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 24 Jul 2024 10:48:18 +0000 Subject: [PATCH 077/123] Fix typo --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3914192f9af..dfdeddb75cf 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2969,8 +2969,8 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - retry( - log_function=lambda exception: logginnfo( + retry_decorator.retry( + log_function=lambda exception: logging.info( f"Azurite initialization failed with error: {exception}" ), )(run_and_check)(azurite_start_cmd) From f21a9e5d0892315ba79d8f10bd9bb6ef271a3aa9 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 24 Jul 2024 11:00:49 +0000 Subject: [PATCH 078/123] Fix bug --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dfdeddb75cf..7d80fbe90f8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2969,7 +2969,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - retry_decorator.retry( + retry( log_function=lambda exception: logging.info( f"Azurite initialization failed with error: {exception}" ), From 2ea10d99940132828c61457d62a54b77a7a66af2 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 24 Jul 2024 13:13:58 +0200 Subject: [PATCH 079/123] Integration tests: fix flaky tests --- .../test_concurrency.py | 11 +++-------- tests/integration/test_manipulate_statistics/test.py | 6 ++++-- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index c08f3c9c242..d3caadd0b46 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -276,15 +276,10 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): for node in nodes: assert_eq_with_retry( node, - f"SELECT status from system.backups WHERE id IN {ids_list} AND (status == 'CREATING_BACKUP')", - "", - ) - - for node in nodes: - assert_eq_with_retry( - node, - f"SELECT status, error from system.backups WHERE id IN {ids_list} AND (status == 'BACKUP_FAILED')", + f"SELECT status from system.backups " + f"WHERE id IN {ids_list} AND ((status == 'CREATING_BACKUP') OR (status == 'BACKUP_FAILED'))", "", + retry_count=100, ) backup_names = {} diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index a602cce63df..2541c9b946f 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -167,7 +167,8 @@ def test_replicated_table_ddl(started_cluster): check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) node1.query( - "ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"} + "ALTER TABLE test_stat CLEAR STATISTICS d", + settings={"alter_sync": "2", "mutations_sync": 2}, ) node1.query( "ALTER TABLE test_stat ADD STATISTICS b type tdigest", @@ -177,7 +178,8 @@ def test_replicated_table_ddl(started_cluster): check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) node1.query( - "ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"} + "ALTER TABLE test_stat MATERIALIZE STATISTICS b", + settings={"alter_sync": "2", "mutations_sync": 2}, ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) From d507adf9a7a524706b8247026acfbb29d8467e2b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Jul 2024 11:33:31 +0000 Subject: [PATCH 080/123] fix --- tests/integration/test_backup_restore_new/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a632e18a650..4806625f3f0 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1239,6 +1239,7 @@ def test_system_users_required_privileges(): instance.query("DROP USER u1") instance.query("DROP ROLE r1") + instance.query("DROP USER u2") def test_system_users_async(): From 595cf9a6a81f034a552d2b0e365bda87e0a7a9f5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 Jul 2024 13:42:32 +0200 Subject: [PATCH 081/123] Revert "Revert "Fix for 992 and friends"" --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- ...00992_system_parts_race_condition_zookeeper_long.sh | 2 +- tests/queries/0_stateless/replication.lib | 10 +++++++++- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3f02486ed15..da379a466af 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3940,7 +3940,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() merge_selecting_task->schedule(); else { - LOG_TRACE(log, "Scheduling next merge selecting task after {}ms", merge_selecting_sleep_ms); + LOG_TRACE(log, "Scheduling next merge selecting task after {}ms, current attempt status: {}", merge_selecting_sleep_ms, result); merge_selecting_task->scheduleAfter(merge_selecting_sleep_ms); } } diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 4887c409844..02a739ece4a 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -41,7 +41,7 @@ function thread3() function thread4() { - while true; do $CLICKHOUSE_CLIENT --receive_timeout=3 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done + while true; do $CLICKHOUSE_CLIENT --receive_timeout=1 -q "OPTIMIZE TABLE alter_table0 FINAL" | grep -Fv "Timeout exceeded while receiving data from server"; done } function thread5() diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 1a86cd9f8db..fe867537000 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -51,6 +51,14 @@ function check_replication_consistency() table_name_prefix=$1 check_query_part=$2 + # Try to kill some mutations because sometimes tests run too much (it's not guarenteed to kill all mutations, see below) + # Try multiple replicas, because queries are not finished yet, and "global" KILL MUTATION may fail due to another query (like DROP TABLE) + readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%'") + for t in "${tables_arr[@]}" + do + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null + done + # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do @@ -96,7 +104,7 @@ function check_replication_consistency() some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||: - # Forcefully cancel mutations to avoid waiting for them to finish + # Forcefully cancel mutations to avoid waiting for them to finish. Kills the remaining mutations ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet From 3f992e4e08c326857c6dec7c5d45fb46154280b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 Jul 2024 13:43:13 +0200 Subject: [PATCH 082/123] Update replication.lib --- tests/queries/0_stateless/replication.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index fe867537000..05651531fba 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -56,7 +56,7 @@ function check_replication_consistency() readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%'") for t in "${tables_arr[@]}" do - ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null ||: done # Wait for all queries to finish (query may still be running if thread is killed by timeout) From 9aaf806021873f010b5d6fda9061ad836de54e36 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 24 Jul 2024 14:03:40 +0200 Subject: [PATCH 083/123] Integration tests: fix flaky tests 2 --- .../test_backup_restore_on_cluster/test_concurrency.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index d3caadd0b46..0e381d48fb1 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -276,7 +276,7 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): for node in nodes: assert_eq_with_retry( node, - f"SELECT status from system.backups " + f"SELECT status, error from system.backups " f"WHERE id IN {ids_list} AND ((status == 'CREATING_BACKUP') OR (status == 'BACKUP_FAILED'))", "", retry_count=100, From 09cb2d822a0960f9917f28d36c47c1e67e861178 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 24 Jul 2024 14:05:50 +0200 Subject: [PATCH 084/123] Change description for setting: `throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert` --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/executeQuery.cpp | 6 +++--- src/Server/TCPHandler.cpp | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index de4725dc350..38ad88cfa1d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -147,7 +147,7 @@ static std::initializer_list executeQueryImpl( /// In case when the client had to retry some mini-INSERTs then they will be properly deduplicated /// by the source tables. This functionality is controlled by a setting `async_insert_deduplicate`. /// But then they will be glued together into a block and pushed through a chain of Materialized Views if any. - /// The process of forming such blocks is not deteministic so each time we retry mini-INSERTs the resulting + /// The process of forming such blocks is not deterministic so each time we retry mini-INSERTs the resulting /// block may be concatenated differently. /// That's why deduplication in dependent Materialized Views doesn't make sense in presence of async INSERTs. if (settings.throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert && settings.deduplicate_blocks_in_dependent_materialized_views) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Deduplication is dependent materialized view cannot work together with async inserts. "\ - "Please disable eiher `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); + "Deduplication in dependent materialized view cannot work together with async inserts. "\ + "Please disable either `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); quota = context->getQuota(); if (quota) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5bc2d09df35..42dc7f5a78f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -961,8 +961,8 @@ void TCPHandler::processInsertQuery() if (settings.throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert && settings.deduplicate_blocks_in_dependent_materialized_views) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Deduplication is dependent materialized view cannot work together with async inserts. "\ - "Please disable eiher `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); + "Deduplication in dependent materialized view cannot work together with async inserts. "\ + "Please disable either `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); auto result = processAsyncInsertQuery(*insert_queue); if (result.status == AsynchronousInsertQueue::PushResult::OK) From d5ea07be2ef2e61edf95408dec70a8e1bc66cafa Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 14:20:03 +0200 Subject: [PATCH 085/123] Bump From 81c19b02e78ea7f2980531732bbaf304622e81fb Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Wed, 24 Jul 2024 13:33:32 +0100 Subject: [PATCH 086/123] Update json.md --- docs/en/sql-reference/data-types/json.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index f2eac12594d..f218c8d0339 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -7,7 +7,7 @@ keywords: [object, data type] # Object Data Type (deprecated) -**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864). +**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-formats/json/overview) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864).
From cb7fafd1e6f04a2f29cd77036bb29042b4cfe3f6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 14:39:26 +0200 Subject: [PATCH 087/123] Fix --- src/Common/StackTrace.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index b2ece02e2f0..59a58ac027a 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -4,7 +4,6 @@ #include #include -#include "Common/PipeFDs.h" #include #include #include @@ -496,8 +495,6 @@ using StackTraceCacheBase = std::map Date: Wed, 24 Jul 2024 14:46:44 +0200 Subject: [PATCH 088/123] 24.7 changelog improvements --- CHANGELOG.md | 34 ++++++++++++++-------------------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4fddd7d7685..a0933bd6544 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,13 +17,12 @@ * `clickhouse-keeper-client` will only accept paths in string literals, such as `ls '/hello/world'`, not bare strings such as `ls /hello/world`. [#65494](https://github.com/ClickHouse/ClickHouse/pull/65494) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). * Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Function `tuple` will now try to construct named tuples in query (controlled by `enable_named_columns_in_function_tuple`). Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). #### New Feature -* Extend function `tuple` to construct named tuples in query. Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). -* A new table function, `fuzzQuery,` was added. This function allows you to modify a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1');`. [#62103](https://github.com/ClickHouse/ClickHouse/pull/62103) ([pufit](https://github.com/pufit)). * Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). -* Support JWT authentication in `clickhouse-client`. [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Support JWT authentication in `clickhouse-client` (will be available only in ClickHouse Cloud). [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). * Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). * Support accept_invalid_certificate in client's config in order to allow for client to connect over secure TCP to a server running with self-signed certificate - can be used as a shorthand for corresponding `openSSL` client settings `verificationMode=none` + `invalidCertificateHandler.name=AcceptCertificateHandler`. [#65238](https://github.com/ClickHouse/ClickHouse/pull/65238) ([peacewalker122](https://github.com/peacewalker122)). @@ -31,15 +30,19 @@ * Add aggregate function `groupConcat`. About the same as `arrayStringConcat( groupArray(column), ',')` Can receive 2 parameters: a string delimiter and the number of elements to be processed. [#65451](https://github.com/ClickHouse/ClickHouse/pull/65451) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Add AzureQueue storage. [#65458](https://github.com/ClickHouse/ClickHouse/pull/65458) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). -* Allow system administrators to configure `logger.console_log_level`. [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). +* Introduce `logger.console_log_level` server config to control the log level to the console (if enabled). [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). * Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). * Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). +* Make an interactive client for clickhouse-disks, add local disk from the local directory. [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). +* When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). #### Experimental Feature * Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). * Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). * Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). * Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). #### Performance Improvement * Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). @@ -51,6 +54,10 @@ * Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). * Unload primary index of outdated parts to reduce total memory usage. [#65852](https://github.com/ClickHouse/ClickHouse/pull/65852) ([Anton Popov](https://github.com/CurtizJ)). * Functions `replaceRegexpAll` and `replaceRegexpOne` are now significantly faster if the pattern is trivial, i.e. contains no metacharacters, pattern classes, flags, grouping characters etc. (Thanks to Taiyang Li). [#66185](https://github.com/ClickHouse/ClickHouse/pull/66185) ([Robert Schulze](https://github.com/rschu1ze)). +* s3 requests: Reduce retry time for queries, increase retries count for backups. 8.5 minutes and 100 retires for queries, 1.2 hours and 1000 retries for backup restore. [#65232](https://github.com/ClickHouse/ClickHouse/pull/65232) ([Sema Checherinda](https://github.com/CheSema)). +* Support query plan LIMIT optimization. Support LIMIT pushdown for PostgreSQL storage and table function. [#65454](https://github.com/ClickHouse/ClickHouse/pull/65454) ([Maksim Kita](https://github.com/kitaisreal)). +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). +* DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). #### Improvement * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -60,21 +67,15 @@ * Allow matching column names in a case insensitive manner when reading json files (`input_format_json_ignore_key_case`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). -* Make an interactive client for clickhouse-disks, add local disk from the local directory. [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). * `StorageS3Queue` related fixes and improvements. Deduce a default value of `s3queue_processing_threads_num` according to the number of physical cpu cores on the server (instead of the previous default value as 1). Set default value of `s3queue_loading_retries` to 10. Fix possible vague "Uncaught exception" in exception column of `system.s3queue`. Do not increment retry count on `MEMORY_LIMIT_EXCEEDED` exception. Move files commit to a stage after insertion into table fully finished to avoid files being commited while not inserted. Add settings `s3queue_max_processed_files_before_commit`, `s3queue_max_processed_rows_before_commit`, `s3queue_max_processed_bytes_before_commit`, `s3queue_max_processing_time_sec_before_commit`, to better control commit and flush time. [#65046](https://github.com/ClickHouse/ClickHouse/pull/65046) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed broken multiple columns aggregation on s390x. [#65062](https://github.com/ClickHouse/ClickHouse/pull/65062) ([Harry Lee](https://github.com/HarryLeeIBM)). * Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). -* s3 requests: Reduce retry time for queries, increase retries count for backups. 8.5 minutes and 100 retires for queries, 1.2 hours and 1000 retries for backup restore. [#65232](https://github.com/ClickHouse/ClickHouse/pull/65232) ([Sema Checherinda](https://github.com/CheSema)). * Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). * Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). -* Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). -* Support query plan LIMIT optimization. Support LIMIT pushdown for PostgreSQL storage and table function. [#65454](https://github.com/ClickHouse/ClickHouse/pull/65454) ([Maksim Kita](https://github.com/kitaisreal)). * `arrayMin`/`arrayMax` can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). * Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). * Do not create format settings for each row when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). -* Fixed out-of-range exception in parsing Dwarf5 on s390x. [#65501](https://github.com/ClickHouse/ClickHouse/pull/65501) ([Harry Lee](https://github.com/HarryLeeIBM)). * Reduce `clickhouse-local` prompt to just `:)`. `getFQDNOrHostName()` takes too long on macOS, and we don't want a hostname in the prompt for `clickhouse-local` anyway. [#65510](https://github.com/ClickHouse/ClickHouse/pull/65510) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Avoid printing a message from jemalloc about per-CPU arenas on low-end virtual machines. [#65532](https://github.com/ClickHouse/ClickHouse/pull/65532) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -82,7 +83,6 @@ * Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). * Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). * Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). -* When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). * Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). * `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add a script to backup your files to ClickHouse. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -93,7 +93,6 @@ * Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). * Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). -* DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). * Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). * Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). * Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). @@ -102,18 +101,16 @@ * Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) -* Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix unexpected size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). * Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). * Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). * Fix the VALID UNTIL clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). - -#### Bug Fix (user-visible misbehavior in an official stable release) * Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). * Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). -* Fix bug with cancelation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fix bug with cancellation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). * Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). * Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). * Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). @@ -146,7 +143,6 @@ * Fix resolving dynamic subcolumns in analyzer, avoid reading the whole column on dynamic subcolumn reading. [#66004](https://github.com/ClickHouse/ClickHouse/pull/66004) ([Kruglov Pavel](https://github.com/Avogar)). * Fix config merging for from_env with replace overrides. [#66034](https://github.com/ClickHouse/ClickHouse/pull/66034) ([Azat Khuzhin](https://github.com/azat)). * Fix a possible hanging in `GRPCServer` during shutdown. [#66061](https://github.com/ClickHouse/ClickHouse/pull/66061) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` peremeter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). * Fixed several cases in function `has` with non-constant `LowCardinality` arguments. [#66088](https://github.com/ClickHouse/ClickHouse/pull/66088) ([Anton Popov](https://github.com/CurtizJ)). * Fix for `groupArrayIntersect`. It had incorrect behavior in the `merge()` function. Also, fixed behavior in `deserialise()` for numeric and general data. [#66103](https://github.com/ClickHouse/ClickHouse/pull/66103) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). @@ -185,8 +181,6 @@ #### Build/Testing/Packaging Improvement * Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). -* Upgraded `pocketfft` dependency to the recent commit https://github.com/mreineck/pocketfft/commit/f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3. [#66291](https://github.com/ClickHouse/ClickHouse/pull/66291) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Upgraded `azure-sdk-for-cpp` to the recent commit https://github.com/ClickHouse/azure-sdk-for-cpp/commit/ea3e19a7be08519134c643177d56c7484dfec884. [#66292](https://github.com/ClickHouse/ClickHouse/pull/66292) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). ### ClickHouse release 24.6, 2024-07-01 From ff44b2066195c8f72897794ff1051a695c56cbfd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Jul 2024 14:51:48 +0200 Subject: [PATCH 089/123] Revert "FuzzQuery table function" --- .../table-functions/fuzzQuery.md | 36 ---- programs/client/Client.h | 5 +- src/Client/ClientBase.h | 2 +- src/{Common => Client}/QueryFuzzer.cpp | 50 ++---- src/{Common => Client}/QueryFuzzer.h | 35 ++-- src/Storages/StorageFuzzQuery.cpp | 169 ------------------ src/Storages/StorageFuzzQuery.h | 88 --------- src/Storages/registerStorages.cpp | 2 - src/TableFunctions/TableFunctionFuzzQuery.cpp | 54 ------ src/TableFunctions/TableFunctionFuzzQuery.h | 42 ----- src/TableFunctions/registerTableFunctions.cpp | 1 - src/TableFunctions/registerTableFunctions.h | 1 - .../03031_table_function_fuzzquery.reference | 2 - .../03031_table_function_fuzzquery.sql | 18 -- 14 files changed, 31 insertions(+), 474 deletions(-) delete mode 100644 docs/en/sql-reference/table-functions/fuzzQuery.md rename src/{Common => Client}/QueryFuzzer.cpp (97%) rename src/{Common => Client}/QueryFuzzer.h (91%) delete mode 100644 src/Storages/StorageFuzzQuery.cpp delete mode 100644 src/Storages/StorageFuzzQuery.h delete mode 100644 src/TableFunctions/TableFunctionFuzzQuery.cpp delete mode 100644 src/TableFunctions/TableFunctionFuzzQuery.h delete mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.reference delete mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.sql diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md deleted file mode 100644 index e15f8a40156..00000000000 --- a/docs/en/sql-reference/table-functions/fuzzQuery.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -slug: /en/sql-reference/table-functions/fuzzQuery -sidebar_position: 75 -sidebar_label: fuzzQuery ---- - -# fuzzQuery - -Perturbs the given query string with random variations. - -``` sql -fuzzQuery(query[, max_query_length[, random_seed]]) -``` - -**Arguments** - -- `query` (String) - The source query to perform the fuzzing on. -- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process. -- `random_seed` (UInt64) - A random seed for producing stable results. - -**Returned Value** - -A table object with a single column containing perturbed query strings. - -## Usage Example - -``` sql -SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2; -``` - -``` - ┌─query──────────────────────────────────────────────────────────┐ -1. │ SELECT 'a' AS key GROUP BY key │ -2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │ - └────────────────────────────────────────────────────────────────┘ -``` diff --git a/programs/client/Client.h b/programs/client/Client.h index 6d57a6ea648..229608f787d 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -9,10 +9,7 @@ namespace DB class Client : public ClientBase { public: - Client() - { - fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr); - } + Client() = default; void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 4f500a4c45d..986990aecaa 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -17,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp similarity index 97% rename from src/Common/QueryFuzzer.cpp rename to src/Client/QueryFuzzer.cpp index 161c38f20e0..f5b700ea529 100644 --- a/src/Common/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -68,21 +68,22 @@ Field QueryFuzzer::getRandomField(int type) { case 0: { - return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)]; + return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) + / sizeof(*bad_int64_values))]; } case 1: { static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, - FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)]; + FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; } case 2: { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % std::size(bad_int64_values)], - static_cast(scales[fuzz_rand() % std::size(scales)]) + bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], + static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) ); } default: @@ -164,8 +165,7 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - if (debug_stream) - *debug_stream << "erased\n"; + std::cerr << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,14 +174,12 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - if (debug_stream) - *debug_stream << fmt::format("inserted (pos {})\n", pos); + std::cerr << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - if (debug_stream) - *debug_stream << "inserted (0)\n"; + std::cerr << "inserted (0)\n"; } } @@ -199,9 +197,7 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - - if (debug_stream) - *debug_stream << "erased\n"; + std::cerr << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -210,16 +206,12 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - - if (debug_stream) - *debug_stream << fmt::format("inserted (pos {})\n", pos); + std::cerr << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - - if (debug_stream) - *debug_stream << "inserted (0)\n"; + std::cerr << "inserted (0)\n"; } } @@ -352,8 +344,7 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - if (debug_stream) - *debug_stream << "No random column.\n"; + std::cerr << "No random column.\n"; } } @@ -387,8 +378,7 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - if (debug_stream) - *debug_stream << "No random column.\n"; + std::cerr << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1371,15 +1361,11 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - if (out_stream) - { - *out_stream << std::endl; - - WriteBufferFromOStream ast_buf(*out_stream, 4096); - formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.finalize(); - *out_stream << std::endl << std::endl; - } + std::cout << std::endl; + WriteBufferFromOStream ast_buf(std::cout, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.finalize(); + std::cout << std::endl << std::endl; } } diff --git a/src/Common/QueryFuzzer.h b/src/Client/QueryFuzzer.h similarity index 91% rename from src/Common/QueryFuzzer.h rename to src/Client/QueryFuzzer.h index 35d088809f2..6165e589cae 100644 --- a/src/Common/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -35,31 +35,9 @@ struct ASTWindowDefinition; * queries, so you want to feed it a lot of queries to get some interesting mix * of them. Normally we feed SQL regression tests to it. */ -class QueryFuzzer +struct QueryFuzzer { -public: - explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) - : fuzz_rand(fuzz_rand_) - , out_stream(out_stream_) - , debug_stream(debug_stream_) - { - } - - // This is the only function you have to call -- it will modify the passed - // ASTPtr to point to new AST with some random changes. - void fuzzMain(ASTPtr & ast); - - ASTs getInsertQueriesForFuzzedTables(const String & full_query); - ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); - void notifyQueryFailed(ASTPtr ast); - - static bool isSuitableForFuzzing(const ASTCreateQuery & create); - -private: - pcg64 fuzz_rand; - - std::ostream * out_stream = nullptr; - std::ostream * debug_stream = nullptr; + pcg64 fuzz_rand{randomSeed()}; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is @@ -88,6 +66,10 @@ private: std::unordered_map index_of_fuzzed_table; std::set created_tables_hashes; + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); @@ -95,6 +77,9 @@ private: ASTPtr getRandomExpressionList(); DataTypePtr fuzzDataType(DataTypePtr type); DataTypePtr getRandomType(); + ASTs getInsertQueriesForFuzzedTables(const String & full_query); + ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); + void notifyQueryFailed(ASTPtr ast); void replaceWithColumnLike(ASTPtr & ast); void replaceWithTableLike(ASTPtr & ast); void fuzzOrderByElement(ASTOrderByElement * elem); @@ -117,6 +102,8 @@ private: void addTableLike(ASTPtr ast); void addColumnLike(ASTPtr ast); void collectFuzzInfoRecurse(ASTPtr ast); + + static bool isSuitableForFuzzing(const ASTCreateQuery & create); }; } diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp deleted file mode 100644 index 6e8f425f8dc..00000000000 --- a/src/Storages/StorageFuzzQuery.cpp +++ /dev/null @@ -1,169 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -ColumnPtr FuzzQuerySource::createColumn() -{ - auto column = ColumnString::create(); - ColumnString::Chars & data_to = column->getChars(); - ColumnString::Offsets & offsets_to = column->getOffsets(); - - offsets_to.resize(block_size); - IColumn::Offset offset = 0; - - auto fuzz_base = query; - size_t row_num = 0; - - while (row_num < block_size) - { - ASTPtr new_query = fuzz_base->clone(); - - auto base_before_fuzz = fuzz_base->formatForErrorMessage(); - fuzzer.fuzzMain(new_query); - auto fuzzed_text = new_query->formatForErrorMessage(); - - if (base_before_fuzz == fuzzed_text) - continue; - - /// AST is too long, will start from the original query. - if (config.max_query_length > 500) - { - fuzz_base = query; - continue; - } - - IColumn::Offset next_offset = offset + fuzzed_text.size() + 1; - data_to.resize(next_offset); - - std::copy(fuzzed_text.begin(), fuzzed_text.end(), &data_to[offset]); - - data_to[offset + fuzzed_text.size()] = 0; - offsets_to[row_num] = next_offset; - - offset = next_offset; - fuzz_base = new_query; - ++row_num; - } - - return column; -} - -StorageFuzzQuery::StorageFuzzQuery( - const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_) - : IStorage(table_id_), config(config_) -{ - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setComment(comment_); - setInMemoryMetadata(storage_metadata); -} - -Pipe StorageFuzzQuery::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /*query_info*/, - ContextPtr /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - storage_snapshot->check(column_names); - - Pipes pipes; - pipes.reserve(num_streams); - - const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); - Block block_header; - for (const auto & name : column_names) - { - const auto & name_type = our_columns.get(name); - MutableColumnPtr column = name_type.type->createColumn(); - block_header.insert({std::move(column), name_type.type, name_type.name}); - } - - const char * begin = config.query.data(); - const char * end = begin + config.query.size(); - - ParserQuery parser(end, false); - auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); - - for (UInt64 i = 0; i < num_streams; ++i) - pipes.emplace_back(std::make_shared(max_block_size, block_header, config, query)); - - return Pipe::unitePipes(std::move(pipes)); -} - -StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) -{ - StorageFuzzQuery::Configuration configuration{}; - - // Supported signatures: - // - // FuzzQuery(query) - // FuzzQuery(query, max_query_length) - // FuzzQuery(query, max_query_length, random_seed) - if (engine_args.empty() || engine_args.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 3 arguments: query, max_query_length, random_seed"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); - configuration.query = std::move(first_arg); - - if (engine_args.size() >= 2) - { - const auto & literal = engine_args[1]->as(); - if (!literal.value.isNull()) - configuration.max_query_length = checkAndGetLiteralArgument(literal, "max_query_length"); - } - - if (engine_args.size() == 3) - { - const auto & literal = engine_args[2]->as(); - if (!literal.value.isNull()) - configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); - } - - return configuration; -} - -void registerStorageFuzzQuery(StorageFactory & factory) -{ - factory.registerStorage( - "FuzzQuery", - [](const StorageFactory::Arguments & args) -> std::shared_ptr - { - ASTs & engine_args = args.engine_args; - - if (engine_args.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage FuzzQuery must have arguments."); - - StorageFuzzQuery::Configuration configuration = StorageFuzzQuery::getConfiguration(engine_args, args.getLocalContext()); - - for (const auto& col : args.columns) - if (col.type->getTypeId() != TypeIndex::String) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "'StorageFuzzQuery' supports only columns of String type, got {}.", col.type->getName()); - - return std::make_shared(args.table_id, args.columns, args.comment, configuration); - }); -} - -} diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h deleted file mode 100644 index 125ef960e74..00000000000 --- a/src/Storages/StorageFuzzQuery.h +++ /dev/null @@ -1,88 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -#include "config.h" - -namespace DB -{ - -class NamedCollection; - -class StorageFuzzQuery final : public IStorage -{ -public: - struct Configuration : public StatelessTableEngineConfiguration - { - String query; - UInt64 max_query_length = 500; - UInt64 random_seed = randomSeed(); - }; - - StorageFuzzQuery( - const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_); - - std::string getName() const override { return "FuzzQuery"; } - - Pipe read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; - - static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); - -private: - const Configuration config; -}; - - -class FuzzQuerySource : public ISource -{ -public: - FuzzQuerySource( - UInt64 block_size_, Block block_header_, const StorageFuzzQuery::Configuration & config_, ASTPtr query_) - : ISource(block_header_) - , block_size(block_size_) - , block_header(std::move(block_header_)) - , config(config_) - , query(query_) - , fuzzer(config_.random_seed) - { - } - - String getName() const override { return "FuzzQuery"; } - -protected: - Chunk generate() override - { - Columns columns; - columns.reserve(block_header.columns()); - for (const auto & col : block_header) - { - chassert(col.type->getTypeId() == TypeIndex::String); - columns.emplace_back(createColumn()); - } - - return {std::move(columns), block_size}; - } - -private: - ColumnPtr createColumn(); - - UInt64 block_size; - Block block_header; - - StorageFuzzQuery::Configuration config; - ASTPtr query; - - QueryFuzzer fuzzer; -}; - -} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index adc1074b1fe..8f33314397c 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -26,7 +26,6 @@ void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); void registerStorageLoop(StorageFactory & factory); -void registerStorageFuzzQuery(StorageFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -127,7 +126,6 @@ void registerStorages() registerStorageExecutable(factory); registerStorageWindowView(factory); registerStorageLoop(factory); - registerStorageFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif diff --git a/src/TableFunctions/TableFunctionFuzzQuery.cpp b/src/TableFunctions/TableFunctionFuzzQuery.cpp deleted file mode 100644 index 224f6666556..00000000000 --- a/src/TableFunctions/TableFunctionFuzzQuery.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include - -#include -#include -#include -#include - -namespace DB -{ - - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -void TableFunctionFuzzQuery::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); - - auto args = args_func.at(0)->children; - configuration = StorageFuzzQuery::getConfiguration(args, context); -} - -StoragePtr TableFunctionFuzzQuery::executeImpl( - const ASTPtr & /*ast_function*/, - ContextPtr context, - const std::string & table_name, - ColumnsDescription /*cached_columns*/, - bool is_insert_query) const -{ - ColumnsDescription columns = getActualTableStructure(context, is_insert_query); - auto res = std::make_shared( - StorageID(getDatabaseName(), table_name), - columns, - /* comment */ String{}, - configuration); - res->startup(); - return res; -} - -void registerTableFunctionFuzzQuery(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description = "Perturbs a query string with random variations.", - .returned_value = "A table object with a single column containing perturbed query strings."}, - .allow_readonly = true}); -} - -} diff --git a/src/TableFunctions/TableFunctionFuzzQuery.h b/src/TableFunctions/TableFunctionFuzzQuery.h deleted file mode 100644 index 22d10341c4d..00000000000 --- a/src/TableFunctions/TableFunctionFuzzQuery.h +++ /dev/null @@ -1,42 +0,0 @@ -#pragma once - -#include - -#include -#include -#include - -#include "config.h" - -namespace DB -{ - -class TableFunctionFuzzQuery : public ITableFunction -{ -public: - static constexpr auto name = "fuzzQuery"; - std::string getName() const override { return name; } - - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - ColumnsDescription getActualTableStructure(ContextPtr /* context */, bool /* is_insert_query */) const override - { - return ColumnsDescription{{"query", std::make_shared()}}; - } - -private: - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "fuzzQuery"; } - - String source; - std::optional random_seed; - StorageFuzzQuery::Configuration configuration; -}; - -} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index a6c90872f12..ca4913898f9 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -26,7 +26,6 @@ void registerTableFunctions() registerTableFunctionMongoDB(factory); registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); - registerTableFunctionFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerTableFunctionFuzzJSON(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 2a8864a9bfd..efde4d6dcdc 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -23,7 +23,6 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMergeTreeIndex(TableFunctionFactory & factory); -void registerTableFunctionFuzzQuery(TableFunctionFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerTableFunctionFuzzJSON(TableFunctionFactory & factory); #endif diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference deleted file mode 100644 index 202e4557a33..00000000000 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference +++ /dev/null @@ -1,2 +0,0 @@ -query -String diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql deleted file mode 100644 index b26096f7f0e..00000000000 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql +++ /dev/null @@ -1,18 +0,0 @@ - -SELECT * FROM fuzzQuery('SELECT 1', 500, 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; - -SELECT * FROM fuzzQuery('SELECT * -FROM ( - SELECT - ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, - count() - FROM numbers(3) - GROUP BY item_id WITH TOTALS -) AS l FULL JOIN ( - SELECT - ([toString((number % 2) * 2)] :: Array(String)) AS item_id - FROM numbers(3) -) AS r -ON l.item_id = r.item_id -ORDER BY 1,2,3; -', 500, 8956) LIMIT 10 FORMAT NULL; From e42e04c5230560f652c75c2c3b43efc3567cb31c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 24 Jul 2024 12:53:52 +0000 Subject: [PATCH 090/123] Test --- ...llel_replicas_alter_select_ubsan.reference | 0 ...5_parallel_replicas_alter_select_ubsan.sql | 35 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.reference create mode 100644 tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.sql diff --git a/tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.reference b/tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.sql b/tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.sql new file mode 100644 index 00000000000..2ec9368327d --- /dev/null +++ b/tests/queries/0_stateless/03205_parallel_replicas_alter_select_ubsan.sql @@ -0,0 +1,35 @@ +SET alter_sync = 2; +SET max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_for_non_replicated_merge_tree = true; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t1__fuzz_26; + +CREATE TABLE t1__fuzz_26 (`a` Nullable(Float64), `b` Nullable(Float32), `pk` Int64) ENGINE = MergeTree ORDER BY pk; +CREATE TABLE t1 ( a Float64, b Int64, pk String) Engine = MergeTree() ORDER BY pk; + +ALTER TABLE t1 + (MODIFY COLUMN `a` Float64 TTL toDateTime(b) + toIntervalMonth(viewExplain('EXPLAIN', 'actions = 1', ( + SELECT + toIntervalMonth(1), + 2 + FROM t1__fuzz_26 + GROUP BY + toFixedString('%Prewhere%', 10), + toNullable(12) + WITH ROLLUP + )), 1)) settings allow_experimental_parallel_reading_from_replicas = 1; -- { serverError INCORRECT_RESULT_OF_SCALAR_SUBQUERY } + +ALTER TABLE t1 + (MODIFY COLUMN `a` Float64 TTL toDateTime(b) + toIntervalMonth(viewExplain('EXPLAIN', 'actions = 1', ( + SELECT + toIntervalMonth(1), + 2 + FROM t1__fuzz_26 + GROUP BY + toFixedString('%Prewhere%', 10), + toNullable(12) + WITH ROLLUP + )), 1)) settings allow_experimental_parallel_reading_from_replicas = 0; -- { serverError INCORRECT_RESULT_OF_SCALAR_SUBQUERY } + +DROP TABLE t1; +DROP TABLE t1__fuzz_26; From 80304b59626c2cf2d9cbe3ba39058b9dfd22ddd9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 15:13:10 +0200 Subject: [PATCH 091/123] Less memory-intensive fill queries for decimal aggregates --- tests/performance/decimal_aggregates.xml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/performance/decimal_aggregates.xml b/tests/performance/decimal_aggregates.xml index ec88be0124f..724d0c5d0e6 100644 --- a/tests/performance/decimal_aggregates.xml +++ b/tests/performance/decimal_aggregates.xml @@ -4,8 +4,13 @@ CREATE TABLE t (x UInt64, d32 Decimal32(3), d64 Decimal64(4), d128 Decimal128(5)) ENGINE = Memory - - INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(500000000) SETTINGS max_threads = 8 + + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000, 100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(200000000, 100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(300000000, 100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(400000000, 100000000) SETTINGS max_threads = 2 + DROP TABLE IF EXISTS t SELECT min(d32), max(d32), argMin(x, d32), argMax(x, d32) FROM t From a03a59a3bfd2dc55f1f04197cad36d147edb3ee1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 15:14:02 +0200 Subject: [PATCH 092/123] Use less data overall --- tests/performance/decimal_aggregates.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/performance/decimal_aggregates.xml b/tests/performance/decimal_aggregates.xml index 724d0c5d0e6..9fc94f01a4d 100644 --- a/tests/performance/decimal_aggregates.xml +++ b/tests/performance/decimal_aggregates.xml @@ -8,8 +8,6 @@ INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000, 100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(200000000, 100000000) SETTINGS max_threads = 2 - INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(300000000, 100000000) SETTINGS max_threads = 2 - INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(400000000, 100000000) SETTINGS max_threads = 2 DROP TABLE IF EXISTS t From 3c6c9303c3046d780ef01ce0fcb87f03b004162a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 24 Jul 2024 13:14:43 +0000 Subject: [PATCH 093/123] disable setting optimize_functions_to_subcolumns --- src/Core/Settings.h | 2 +- tests/clickhouse-test | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3d181e33001..fbb7663b612 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -602,7 +602,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ - M(Bool, optimize_functions_to_subcolumns, true, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ + M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 0c04d8fb2c3..a29c786e998 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,6 +835,7 @@ class SettingsRandomizer: "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), + "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), } @staticmethod From 5d14d823e3882887ab8238c6ee5778f552cce835 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 24 Jul 2024 13:27:19 +0000 Subject: [PATCH 094/123] remove setting from changes history --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index de4725dc350..c395bfdc815 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -63,7 +63,6 @@ static std::initializer_list Date: Wed, 24 Jul 2024 15:33:59 +0200 Subject: [PATCH 095/123] Bump From 535c872b39b7f593e9e63e614a78d39cde5dd4d9 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Wed, 24 Jul 2024 15:46:48 +0200 Subject: [PATCH 096/123] empty comit From c5d262c23b4ee289a374ae5d817a101bd44154c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 24 Jul 2024 13:55:59 +0000 Subject: [PATCH 097/123] Increase max allocation size for sanitizers --- docker/test/base/Dockerfile | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 2317f84e0cb..79ed871b822 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -23,15 +23,16 @@ RUN apt-get update \ # and MEMORY_LIMIT_EXCEEDED exceptions in Functional tests (total memory limit in Functional tests is ~55.24 GiB). # TSAN will flush shadow memory when reaching this limit. # It may cause false-negatives, but it's better than OOM. -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1'" >> /etc/environment -RUN echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment -RUN echo "MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'" >> /etc/environment -RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment +# max_allocation_size_mb is set to 32GB, so we have much bigger chance to run into memory limit than the limitation of the sanitizers +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1 max_allocation_size_mb=32768'" >> /etc/environment +RUN echo "UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768'" >> /etc/environment +RUN echo "MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768'" >> /etc/environment +RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt max_allocation_size_mb=32768'" >> /etc/environment # Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") # (but w/o verbosity for TSAN, otherwise test.reference will not match) -ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' -ENV UBSAN_OPTIONS='print_stacktrace=1' -ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' +ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1 max_allocation_size_mb=32768' +ENV UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768' +ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768' # for external_symbolizer_path RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer From e498b86f50dbba8ac8e50f832547850c7cc29c6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 24 Jul 2024 14:00:28 +0000 Subject: [PATCH 098/123] Add option for LSAN in shell also --- docker/test/base/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 79ed871b822..a81826ed6b5 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -33,6 +33,7 @@ RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_supp ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1 max_allocation_size_mb=32768' ENV UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768' +ENV LSAN_OPTIONS='max_allocation_size_mb=32768' # for external_symbolizer_path RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer From 62d956f8a882f74a930340cce1650babc13bf7a1 Mon Sep 17 00:00:00 2001 From: zoomxi <419486879@qq.com> Date: Wed, 24 Jul 2024 22:39:30 +0800 Subject: [PATCH 099/123] remove unused file ParallelReplicasReadingCoordinator.h --- src/Interpreters/ClusterProxy/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 59f095f7487..d04a73e384e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include From 4ab4b6729224806e7d34f1bee91a21dcd1818e8f Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Wed, 24 Jul 2024 07:59:34 -0700 Subject: [PATCH 100/123] Update link for JSON schema inference --- docs/en/getting-started/example-datasets/nypd_complaint_data.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/nypd_complaint_data.md b/docs/en/getting-started/example-datasets/nypd_complaint_data.md index a178fe456a6..1caa1ab6a9a 100644 --- a/docs/en/getting-started/example-datasets/nypd_complaint_data.md +++ b/docs/en/getting-started/example-datasets/nypd_complaint_data.md @@ -55,7 +55,7 @@ CMPLNT_FR_TM Nullable(String) ``` :::tip -Most of the time the above command will let you know which fields in the input data are numeric, and which are strings, and which are tuples. This is not always the case. Because ClickHouse is routineley used with datasets containing billions of records there is a default number (100) of rows examined to [infer the schema](/docs/en/integrations/data-ingestion/data-formats/json.md#relying-on-schema-inference) in order to avoid parsing billions of rows to infer the schema. The response below may not match what you see, as the dataset is updated several times each year. Looking at the Data Dictionary you can see that CMPLNT_NUM is specified as text, and not numeric. By overriding the default of 100 rows for inference with the setting `SETTINGS input_format_max_rows_to_read_for_schema_inference=2000` +Most of the time the above command will let you know which fields in the input data are numeric, and which are strings, and which are tuples. This is not always the case. Because ClickHouse is routineley used with datasets containing billions of records there is a default number (100) of rows examined to [infer the schema](/en/integrations/data-formats/json/inference) in order to avoid parsing billions of rows to infer the schema. The response below may not match what you see, as the dataset is updated several times each year. Looking at the Data Dictionary you can see that CMPLNT_NUM is specified as text, and not numeric. By overriding the default of 100 rows for inference with the setting `SETTINGS input_format_max_rows_to_read_for_schema_inference=2000` you can get a better idea of the content. Note: as of version 22.5 the default is now 25,000 rows for inferring the schema, so only change the setting if you are on an older version or if you need more than 25,000 rows to be sampled. From f990f235da10b72e1625d007563761da52067753 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 17:20:19 +0200 Subject: [PATCH 101/123] Increase timeout for test_broken_part_during_merge --- tests/integration/test_broken_part_during_merge/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_broken_part_during_merge/test.py b/tests/integration/test_broken_part_during_merge/test.py index 19c22201fb0..0ba7beeb1fd 100644 --- a/tests/integration/test_broken_part_during_merge/test.py +++ b/tests/integration/test_broken_part_during_merge/test.py @@ -54,7 +54,7 @@ def test_merge_and_part_corruption(started_cluster): with Pool(1) as p: def optimize_with_delay(x): - node1.query("OPTIMIZE TABLE replicated_mt FINAL", timeout=30) + node1.query("OPTIMIZE TABLE replicated_mt FINAL", timeout=120) # corrupt part after merge already assigned, but not started res_opt = p.apply_async(optimize_with_delay, (1,)) @@ -70,7 +70,7 @@ def test_merge_and_part_corruption(started_cluster): node1.query( "ALTER TABLE replicated_mt UPDATE value = 7 WHERE 1", settings={"mutations_sync": 2}, - timeout=30, + timeout=120, ) assert node1.query("SELECT sum(value) FROM replicated_mt") == "2100000\n" From 1cca1d4ba3c9f020f21e684423ace943d863bb0f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 17:27:18 +0200 Subject: [PATCH 102/123] bump From 8d4b919bf4d02c09399296e5213525f78bd68a21 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 24 Jul 2024 17:45:40 +0200 Subject: [PATCH 103/123] Fix detaching broken parts from backup. --- .../MergeTree/DataPartStorageOnDiskBase.cpp | 37 ++++++++++++++++--- .../MergeTree/DataPartStorageOnDiskBase.h | 3 ++ 2 files changed, 34 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 378a1944396..df151e8478f 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -73,12 +73,7 @@ std::optional DataPartStorageOnDiskBase::getRelativePathForPrefix(Logger for (int try_no = 0; try_no < 10; ++try_no) { - if (prefix.empty()) - res = part_dir + (try_no ? "_try" + DB::toString(try_no) : ""); - else if (prefix.ends_with("_")) - res = prefix + part_dir + (try_no ? "_try" + DB::toString(try_no) : ""); - else - res = prefix + "_" + part_dir + (try_no ? "_try" + DB::toString(try_no) : ""); + res = getPartDirForPrefix(prefix, detached, try_no); if (!volume->getDisk()->exists(full_relative_path / res)) return res; @@ -101,6 +96,36 @@ std::optional DataPartStorageOnDiskBase::getRelativePathForPrefix(Logger return res; } +String DataPartStorageOnDiskBase::getPartDirForPrefix(const String & prefix, bool detached, int try_no) const +{ + /// This function joins `prefix` and the part name and an attempt number returning something like "__". + String res = prefix; + if (!prefix.empty() && !prefix.ends_with("_")) + res += "_"; + + /// During RESTORE temporary part directories are created with names like "tmp_restore_all_2_2_0-XXXXXXXX". + /// To detach such a directory we need to rename it replacing "tmp_restore_" with a specified prefix, + /// and a random suffix with an attempt number. + String part_name; + if (detached && part_dir.starts_with("tmp_restore_")) + { + part_name = part_dir.substr(strlen("tmp_restore_")); + size_t endpos = part_name.find('-'); + if (endpos != String::npos) + part_name.erase(endpos, String::npos); + } + + if (!part_name.empty()) + res += part_name; + else + res += part_dir; + + if (try_no) + res += "_try" + DB::toString(try_no); + + return res; +} + bool DataPartStorageOnDiskBase::looksLikeBrokenDetachedPartHasTheSameContent(const String & detached_part_path, std::optional & original_checksums_content, std::optional & original_files_list) const diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 81353d4e20b..1707efc2d4d 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -148,6 +148,9 @@ private: /// Actual file name may be the same as expected /// or be the name of the file with packed data. virtual NameSet getActualFileNamesOnDisk(const NameSet & file_names) const = 0; + + /// Returns the destination path for the part directory while copying a detached part. + String getPartDirForPrefix(const String & prefix, bool detached, int try_no) const; }; } From fa48ff0ca1561f20eb5abb285f400a8faf664d0b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 24 Jul 2024 15:49:17 +0000 Subject: [PATCH 104/123] Separate tests --- tests/integration/test_storage_s3/test.py | 52 ++++++++++------------- 1 file changed, 22 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index d13605170ec..dae98daf0cf 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1148,31 +1148,27 @@ def test_url_reconnect_in_the_middle(started_cluster): assert result == "1000000\t3914219105369203805\n" -def test_seekable_formats(started_cluster): - bucket = started_cluster.minio_bucket + +# At the time of writing the actual read bytes are respectively 148 and 169, so -10% to not be flaky +@pytest.mark.parametrize("format_name,expected_bytes_read", [("Parquet", 133), ("ORC", 150)]) +def test_seekable_formats(started_cluster, format_name, expected_bytes_read): + expected_lines=1500000 instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" + table_function = f"s3(s3_{format_name.lower()}, structure='a Int32, b String', format='{format_name}')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", - timeout=100, + f"INSERT INTO TABLE FUNCTION {table_function} SELECT number, randomString(100) FROM numbers({expected_lines}) settings s3_truncate_on_insert=1", + timeout=300, ) result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 1000000 - - table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" - exec_query_with_retry( - instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", - timeout=100, - ) + assert int(result) == expected_lines result = instance.query( f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='60M', max_download_threads=1" ) - assert int(result) == 1500000 + assert int(result) == expected_lines instance.query(f"SELECT * FROM {table_function} FORMAT Null") @@ -1183,35 +1179,31 @@ def test_seekable_formats(started_cluster): result = result.strip() assert result.endswith("MiB") result = result[: result.index(".")] - assert int(result) > 150 + assert int(result) > 140 -def test_seekable_formats_url(started_cluster): +@pytest.mark.parametrize("format_name", ["Parquet", "ORC"]) +def test_seekable_formats_url(started_cluster, format_name): bucket = started_cluster.minio_bucket + expected_lines=1500000 instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" + format_name_lower=format_name.lower() + table_function = f"s3(s3_{format_name_lower}, structure='a Int32, b String', format='{format_name}')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", - timeout=100, + f"INSERT INTO TABLE FUNCTION {table_function} SELECT number, randomString(100) FROM numbers({expected_lines}) settings s3_truncate_on_insert=1", + timeout=300, ) result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 1500000 + assert int(result) == expected_lines - table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" - exec_query_with_retry( - instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", - timeout=100, - ) - - table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" + url_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_{format_name_lower}', '{format_name}', 'a Int32, b String')" result = instance.query( - f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='60M'" + f"SELECT count() FROM {url_function} SETTINGS max_memory_usage='60M'" ) - assert int(result) == 1500000 + assert int(result) == expected_lines def test_empty_file(started_cluster): From 68c53764f271463bf7d05639c7d31975717e8d6f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 17:50:42 +0200 Subject: [PATCH 105/123] Update tests/performance/decimal_aggregates.xml --- tests/performance/decimal_aggregates.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/decimal_aggregates.xml b/tests/performance/decimal_aggregates.xml index 9fc94f01a4d..b204dddea6e 100644 --- a/tests/performance/decimal_aggregates.xml +++ b/tests/performance/decimal_aggregates.xml @@ -8,6 +8,7 @@ INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000, 100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(200000000, 100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(300000000, 100000000) SETTINGS max_threads = 2 DROP TABLE IF EXISTS t From 2e25808e586d46baf52029d626d02fdcc6ff53d2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 24 Jul 2024 18:03:48 +0200 Subject: [PATCH 106/123] Fix test test_backup_restore_s3. --- tests/integration/test_backup_restore_s3/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 4840f5afc66..381268ce7fe 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -254,6 +254,7 @@ def check_system_tables(backup_query_id=None): ("disk_s3_cache", "ObjectStorage", "S3", "Local"), ("disk_s3_other_bucket", "ObjectStorage", "S3", "Local"), ("disk_s3_plain", "ObjectStorage", "S3", "Plain"), + ("disk_s3_plain_rewritable", "ObjectStorage", "S3", "PlainRewritable"), ("disk_s3_restricted_user", "ObjectStorage", "S3", "Local"), ) assert len(expected_disks) == len(disks) From ce153c9b54045959877e0b400971db352fbeb916 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Jul 2024 16:05:10 +0000 Subject: [PATCH 107/123] Automatic style fix --- tests/integration/test_storage_s3/test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index dae98daf0cf..ab327afe90b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1148,11 +1148,12 @@ def test_url_reconnect_in_the_middle(started_cluster): assert result == "1000000\t3914219105369203805\n" - # At the time of writing the actual read bytes are respectively 148 and 169, so -10% to not be flaky -@pytest.mark.parametrize("format_name,expected_bytes_read", [("Parquet", 133), ("ORC", 150)]) +@pytest.mark.parametrize( + "format_name,expected_bytes_read", [("Parquet", 133), ("ORC", 150)] +) def test_seekable_formats(started_cluster, format_name, expected_bytes_read): - expected_lines=1500000 + expected_lines = 1500000 instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_function = f"s3(s3_{format_name.lower()}, structure='a Int32, b String', format='{format_name}')" @@ -1185,10 +1186,10 @@ def test_seekable_formats(started_cluster, format_name, expected_bytes_read): @pytest.mark.parametrize("format_name", ["Parquet", "ORC"]) def test_seekable_formats_url(started_cluster, format_name): bucket = started_cluster.minio_bucket - expected_lines=1500000 + expected_lines = 1500000 instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - format_name_lower=format_name.lower() + format_name_lower = format_name.lower() table_function = f"s3(s3_{format_name_lower}, structure='a Int32, b String', format='{format_name}')" exec_query_with_retry( instance, From 67567fcff481793cf7828808c094221ad2ec7389 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 19:51:22 +0200 Subject: [PATCH 108/123] Fix build --- programs/odbc-bridge/tests/CMakeLists.txt | 2 +- src/Common/examples/CMakeLists.txt | 38 ++++++++--------- src/Common/mysqlxx/tests/CMakeLists.txt | 2 +- src/Compression/examples/CMakeLists.txt | 2 +- src/Core/examples/CMakeLists.txt | 4 +- src/IO/examples/CMakeLists.txt | 50 +++++++++++------------ src/Interpreters/examples/CMakeLists.txt | 20 ++++----- src/Parsers/examples/CMakeLists.txt | 2 +- utils/corrector_utf8/CMakeLists.txt | 2 +- 9 files changed, 61 insertions(+), 61 deletions(-) diff --git a/programs/odbc-bridge/tests/CMakeLists.txt b/programs/odbc-bridge/tests/CMakeLists.txt index edf364ea192..f1411dbb554 100644 --- a/programs/odbc-bridge/tests/CMakeLists.txt +++ b/programs/odbc-bridge/tests/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp ../validateODBCConnectionString.cpp) -target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io) +target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io clickhouse_common_config) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index c133e9f5617..69580d4ad0e 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -1,14 +1,14 @@ clickhouse_add_executable (hashes_test hashes_test.cpp) -target_link_libraries (hashes_test PRIVATE clickhouse_common_io ch_contrib::cityhash) +target_link_libraries (hashes_test PRIVATE clickhouse_common_io clickhouse_common_config ch_contrib::cityhash) if (TARGET OpenSSL::Crypto) target_link_libraries (hashes_test PRIVATE OpenSSL::Crypto) endif() clickhouse_add_executable (sip_hash_perf sip_hash_perf.cpp) -target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) +target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (small_table small_table.cpp) -target_link_libraries (small_table PRIVATE clickhouse_common_io) +target_link_libraries (small_table PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (parallel_aggregation parallel_aggregation.cpp) target_link_libraries (parallel_aggregation PRIVATE dbms clickhouse_functions) @@ -17,13 +17,13 @@ clickhouse_add_executable (parallel_aggregation2 parallel_aggregation2.cpp) target_link_libraries (parallel_aggregation2 PRIVATE dbms clickhouse_functions) clickhouse_add_executable (int_hashes_perf int_hashes_perf.cpp) -target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io) +target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (compact_array compact_array.cpp) -target_link_libraries (compact_array PRIVATE clickhouse_common_io) +target_link_libraries (compact_array PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (radix_sort radix_sort.cpp) -target_link_libraries (radix_sort PRIVATE clickhouse_common_io ch_contrib::pdqsort) +target_link_libraries (radix_sort PRIVATE clickhouse_common_io clickhouse_common_config ch_contrib::pdqsort) clickhouse_add_executable (arena_with_free_lists arena_with_free_lists.cpp) target_link_libraries (arena_with_free_lists PRIVATE dbms) @@ -33,46 +33,46 @@ target_link_libraries (lru_hash_map_perf PRIVATE dbms) if (OS_LINUX) clickhouse_add_executable (thread_creation_latency thread_creation_latency.cpp) - target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) + target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io clickhouse_common_config) endif() clickhouse_add_executable (array_cache array_cache.cpp) -target_link_libraries (array_cache PRIVATE clickhouse_common_io) +target_link_libraries (array_cache PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (space_saving space_saving.cpp) -target_link_libraries (space_saving PRIVATE clickhouse_common_io) +target_link_libraries (space_saving PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (integer_hash_tables_benchmark integer_hash_tables_benchmark.cpp) target_link_libraries (integer_hash_tables_benchmark PRIVATE dbms ch_contrib::abseil_swiss_tables ch_contrib::sparsehash) clickhouse_add_executable (cow_columns cow_columns.cpp) -target_link_libraries (cow_columns PRIVATE clickhouse_common_io) +target_link_libraries (cow_columns PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (cow_compositions cow_compositions.cpp) -target_link_libraries (cow_compositions PRIVATE clickhouse_common_io) +target_link_libraries (cow_compositions PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (stopwatch stopwatch.cpp) -target_link_libraries (stopwatch PRIVATE clickhouse_common_io) +target_link_libraries (stopwatch PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (symbol_index symbol_index.cpp) -target_link_libraries (symbol_index PRIVATE clickhouse_common_io) +target_link_libraries (symbol_index PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (chaos_sanitizer chaos_sanitizer.cpp) -target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io) +target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io clickhouse_common_config) if (OS_LINUX) clickhouse_add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) - target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) + target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io clickhouse_common_config) endif() clickhouse_add_executable (procfs_metrics_provider_perf procfs_metrics_provider_perf.cpp) -target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io) +target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (average average.cpp) -target_link_libraries (average PRIVATE clickhouse_common_io) +target_link_libraries (average PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (shell_command_inout shell_command_inout.cpp) -target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) +target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (executable_udf executable_udf.cpp) target_link_libraries (executable_udf PRIVATE dbms) @@ -91,4 +91,4 @@ if (ENABLE_SSL) endif() clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp) -target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io) +target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io clickhouse_common_config) diff --git a/src/Common/mysqlxx/tests/CMakeLists.txt b/src/Common/mysqlxx/tests/CMakeLists.txt index c560d0e3153..f62908ddcaf 100644 --- a/src/Common/mysqlxx/tests/CMakeLists.txt +++ b/src/Common/mysqlxx/tests/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (mysqlxx_pool_test mysqlxx_pool_test.cpp) -target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx) +target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx clickhouse_common_config) diff --git a/src/Compression/examples/CMakeLists.txt b/src/Compression/examples/CMakeLists.txt index a7cc6bebf42..fee8cf89942 100644 --- a/src/Compression/examples/CMakeLists.txt +++ b/src/Compression/examples/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE clickhouse_common_io clickhouse_compression) +target_link_libraries (compressed_buffer PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression) diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index f30ee25491f..97e5e8e67e6 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -1,8 +1,8 @@ clickhouse_add_executable (string_pool string_pool.cpp) -target_link_libraries (string_pool PRIVATE clickhouse_common_io ch_contrib::sparsehash) +target_link_libraries (string_pool PRIVATE clickhouse_common_io clickhouse_common_config ch_contrib::sparsehash) clickhouse_add_executable (field field.cpp) target_link_libraries (field PRIVATE dbms) clickhouse_add_executable (string_ref_hash string_ref_hash.cpp) -target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) +target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io clickhouse_common_config) diff --git a/src/IO/examples/CMakeLists.txt b/src/IO/examples/CMakeLists.txt index fc9d9c7dcd1..bfd171a3d00 100644 --- a/src/IO/examples/CMakeLists.txt +++ b/src/IO/examples/CMakeLists.txt @@ -1,77 +1,77 @@ clickhouse_add_executable (read_buffer read_buffer.cpp) -target_link_libraries (read_buffer PRIVATE clickhouse_common_io) +target_link_libraries (read_buffer PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (read_buffer_perf read_buffer_perf.cpp) -target_link_libraries (read_buffer_perf PRIVATE clickhouse_common_io) +target_link_libraries (read_buffer_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (read_float_perf read_float_perf.cpp) -target_link_libraries (read_float_perf PRIVATE clickhouse_common_io) +target_link_libraries (read_float_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (write_buffer write_buffer.cpp) -target_link_libraries (write_buffer PRIVATE clickhouse_common_io) +target_link_libraries (write_buffer PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (write_buffer_perf write_buffer_perf.cpp) -target_link_libraries (write_buffer_perf PRIVATE clickhouse_common_io) +target_link_libraries (write_buffer_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (valid_utf8_perf valid_utf8_perf.cpp) -target_link_libraries (valid_utf8_perf PRIVATE clickhouse_common_io) +target_link_libraries (valid_utf8_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (valid_utf8 valid_utf8.cpp) -target_link_libraries (valid_utf8 PRIVATE clickhouse_common_io) +target_link_libraries (valid_utf8 PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (var_uint var_uint.cpp) -target_link_libraries (var_uint PRIVATE clickhouse_common_io) +target_link_libraries (var_uint PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (read_escaped_string read_escaped_string.cpp) -target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io) +target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (parse_int_perf parse_int_perf.cpp) -target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io) +target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (parse_int_perf2 parse_int_perf2.cpp) -target_link_libraries (parse_int_perf2 PRIVATE clickhouse_common_io) +target_link_libraries (parse_int_perf2 PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (read_write_int read_write_int.cpp) -target_link_libraries (read_write_int PRIVATE clickhouse_common_io) +target_link_libraries (read_write_int PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (o_direct_and_dirty_pages o_direct_and_dirty_pages.cpp) -target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io) +target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (io_operators io_operators.cpp) -target_link_libraries (io_operators PRIVATE clickhouse_common_io) +target_link_libraries (io_operators PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (write_int write_int.cpp) -target_link_libraries (write_int PRIVATE clickhouse_common_io) +target_link_libraries (write_int PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (zlib_buffers zlib_buffers.cpp) -target_link_libraries (zlib_buffers PRIVATE clickhouse_common_io) +target_link_libraries (zlib_buffers PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (lzma_buffers lzma_buffers.cpp) -target_link_libraries (lzma_buffers PRIVATE clickhouse_common_io) +target_link_libraries (lzma_buffers PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (limit_read_buffer limit_read_buffer.cpp) -target_link_libraries (limit_read_buffer PRIVATE clickhouse_common_io) +target_link_libraries (limit_read_buffer PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (limit_read_buffer2 limit_read_buffer2.cpp) -target_link_libraries (limit_read_buffer2 PRIVATE clickhouse_common_io) +target_link_libraries (limit_read_buffer2 PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (parse_date_time_best_effort parse_date_time_best_effort.cpp) -target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io) +target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (zlib_ng_bug zlib_ng_bug.cpp) -target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib clickhouse_common_io) +target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (dragonbox_test dragonbox_test.cpp) -target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars clickhouse_common_io) +target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (zstd_buffers zstd_buffers.cpp) -target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) +target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (snappy_read_buffer snappy_read_buffer.cpp) -target_link_libraries (snappy_read_buffer PRIVATE clickhouse_common_io) +target_link_libraries (snappy_read_buffer PRIVATE clickhouse_common_io clickhouse_common_config) clickhouse_add_executable (hadoop_snappy_read_buffer hadoop_snappy_read_buffer.cpp) -target_link_libraries (hadoop_snappy_read_buffer PRIVATE clickhouse_common_io) +target_link_libraries (hadoop_snappy_read_buffer PRIVATE clickhouse_common_io clickhouse_common_config) if (TARGET ch_contrib::hdfs) clickhouse_add_executable (read_buffer_from_hdfs read_buffer_from_hdfs.cpp) diff --git a/src/Interpreters/examples/CMakeLists.txt b/src/Interpreters/examples/CMakeLists.txt index 8bb7f9eeb98..4b1ec970b26 100644 --- a/src/Interpreters/examples/CMakeLists.txt +++ b/src/Interpreters/examples/CMakeLists.txt @@ -2,34 +2,34 @@ clickhouse_add_executable (hash_map hash_map.cpp) target_link_libraries (hash_map PRIVATE dbms clickhouse_functions ch_contrib::sparsehash) clickhouse_add_executable (hash_map_lookup hash_map_lookup.cpp) -target_link_libraries (hash_map_lookup PRIVATE clickhouse_common_io clickhouse_compression) +target_link_libraries (hash_map_lookup PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression) clickhouse_add_executable (hash_map3 hash_map3.cpp) -target_link_libraries (hash_map3 PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::farmhash ch_contrib::metrohash) +target_link_libraries (hash_map3 PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::farmhash ch_contrib::metrohash) clickhouse_add_executable (hash_map_string hash_map_string.cpp) -target_link_libraries (hash_map_string PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::sparsehash) +target_link_libraries (hash_map_string PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::sparsehash) clickhouse_add_executable (hash_map_string_2 hash_map_string_2.cpp) -target_link_libraries (hash_map_string_2 PRIVATE clickhouse_common_io clickhouse_compression) +target_link_libraries (hash_map_string_2 PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression) clickhouse_add_executable (hash_map_string_3 hash_map_string_3.cpp) -target_link_libraries (hash_map_string_3 PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::farmhash ch_contrib::metrohash) +target_link_libraries (hash_map_string_3 PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::farmhash ch_contrib::metrohash) clickhouse_add_executable (hash_map_string_small hash_map_string_small.cpp) -target_link_libraries (hash_map_string_small PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::sparsehash) +target_link_libraries (hash_map_string_small PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::sparsehash) clickhouse_add_executable (string_hash_map string_hash_map.cpp) -target_link_libraries (string_hash_map PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::sparsehash) +target_link_libraries (string_hash_map PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::sparsehash) clickhouse_add_executable (string_hash_map_aggregation string_hash_map.cpp) -target_link_libraries (string_hash_map_aggregation PRIVATE clickhouse_common_io clickhouse_compression) +target_link_libraries (string_hash_map_aggregation PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression) clickhouse_add_executable (string_hash_set string_hash_set.cpp) -target_link_libraries (string_hash_set PRIVATE clickhouse_common_io clickhouse_compression) +target_link_libraries (string_hash_set PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression) clickhouse_add_executable (two_level_hash_map two_level_hash_map.cpp) -target_link_libraries (two_level_hash_map PRIVATE clickhouse_common_io clickhouse_compression ch_contrib::sparsehash) +target_link_libraries (two_level_hash_map PRIVATE clickhouse_common_io clickhouse_common_config clickhouse_compression ch_contrib::sparsehash) clickhouse_add_executable (jit_example jit_example.cpp) target_link_libraries (jit_example PRIVATE dbms) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 261f234081c..07f60601acd 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -1,7 +1,7 @@ set(SRCS) clickhouse_add_executable(lexer lexer.cpp ${SRCS}) -target_link_libraries(lexer PRIVATE clickhouse_parsers) +target_link_libraries(lexer PRIVATE clickhouse_parsers clickhouse_common_config) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") target_link_libraries(select_parser PRIVATE dbms) diff --git a/utils/corrector_utf8/CMakeLists.txt b/utils/corrector_utf8/CMakeLists.txt index 4744dd5e0a5..17f047a8cff 100644 --- a/utils/corrector_utf8/CMakeLists.txt +++ b/utils/corrector_utf8/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(corrector_utf8 corrector_utf8.cpp) -target_link_libraries(corrector_utf8 PRIVATE clickhouse_common_io) +target_link_libraries(corrector_utf8 PRIVATE clickhouse_common_io clickhouse_common_config) From 9de43325e4771f54e29ccc43722c5f8c104f129a Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 24 Jul 2024 19:51:34 +0200 Subject: [PATCH 109/123] CI: Jepsen Workflow with CI buddy --- .github/workflows/jepsen.yml | 69 +++++++++++++++++++++++++++++------- tests/ci/ci.py | 40 +++++++++++++++------ tests/ci/ci_config.py | 26 ++++++++++++-- tests/ci/ci_definitions.py | 8 +++++ tests/ci/ci_settings.py | 7 ---- tests/ci/jepsen_check.py | 48 ++++++------------------- tests/ci/pr_info.py | 4 +++ tests/ci/test_ci_config.py | 26 ++++++++++++++ 8 files changed, 159 insertions(+), 69 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index db837ac1ec7..035ba2e5b98 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -9,19 +9,64 @@ on: # yamllint disable-line rule:truthy - cron: '0 */6 * * *' workflow_dispatch: jobs: + RunConfig: + runs-on: [self-hosted, style-checker-aarch64] + outputs: + data: ${{ steps.runconfig.outputs.CI_DATA }} + steps: + - name: DebugInfo + uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true # to ensure correct digests + fetch-depth: 0 # to get version + filter: tree:0 + - name: PrepareRunConfig + id: runconfig + run: | + echo "::group::configure CI run" + python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --workflow "$GITHUB_WORKFLOW" --outfile ${{ runner.temp }}/ci_run_data.json + echo "::endgroup::" + + echo "::group::CI run configure results" + python3 -m json.tool ${{ runner.temp }}/ci_run_data.json + echo "::endgroup::" + { + echo 'CI_DATA<> "$GITHUB_OUTPUT" KeeperJepsenRelease: - uses: ./.github/workflows/reusable_simple_job.yml + needs: [RunConfig] + uses: ./.github/workflows/reusable_test.yml with: - test_name: Jepsen keeper check - runner_type: style-checker - report_required: true + test_name: ClickHouse Keeper Jepsen + runner_type: style-checker-aarch64 + data: ${{ needs.RunConfig.outputs.data }} run_command: | python3 jepsen_check.py keeper - # ServerJepsenRelease: - # uses: ./.github/workflows/reusable_simple_job.yml - # with: - # test_name: Jepsen server check - # runner_type: style-checker - # run_command: | - # cd "$REPO_COPY/tests/ci" - # python3 jepsen_check.py server + ServerJepsenRelease: + if: false # skip for server + needs: [RunConfig] + uses: ./.github/workflows/reusable_test.yml + with: + test_name: ClickHouse Server Jepsen + runner_type: style-checker-aarch64 + data: ${{ needs.RunConfig.outputs.data }} + run_command: | + python3 jepsen_check.py server + CheckWorkflow: + if: ${{ !cancelled() }} + needs: [RunConfig, ServerJepsenRelease, KeeperJepsenRelease] + runs-on: [self-hosted, style-checker-aarch64] + steps: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + - name: Check Workflow results + run: | + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ toJson(needs) }} + EOF + python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 171819e2632..e30062c32ff 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -95,6 +95,12 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace: action="store_true", help="Action that configures ci run. Calculates digests, checks job to be executed, generates json output", ) + parser.add_argument( + "--workflow", + default="", + type=str, + help="Workflow Name, to be provided with --configure for workflow-specific CI runs", + ) parser.add_argument( "--update-gh-statuses", action="store_true", @@ -287,7 +293,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info): # for release/master branches reports must be from the same branch report_prefix = "" if pr_info.is_master or pr_info.is_release: - report_prefix = normalize_string(pr_info.head_ref) + # do not set report prefix for scheduled or dispatched wf (in case it started from feature branch while + # testing), otherwise reports won't be found + if not (pr_info.is_scheduled or pr_info.is_dispatched): + report_prefix = normalize_string(pr_info.head_ref) print( f"Use report prefix [{report_prefix}], pr_num [{pr_info.number}], head_ref [{pr_info.head_ref}]" ) @@ -520,6 +529,7 @@ def _configure_jobs( pr_info: PRInfo, ci_settings: CiSettings, skip_jobs: bool, + workflow_name: str = "", dry_run: bool = False, ) -> CiCache: """ @@ -537,18 +547,27 @@ def _configure_jobs( is_docs_only=pr_info.has_changes_in_documentation_only(), is_master=pr_info.is_master, is_pr=pr_info.is_pr, + workflow_name=workflow_name, ) else: job_configs = {} - # filter jobs in accordance with ci settings - job_configs = ci_settings.apply( - job_configs, - pr_info.is_release, - is_pr=pr_info.is_pr, - is_mq=pr_info.is_merge_queue, - labels=pr_info.labels, - ) + if not workflow_name: + # filter jobs in accordance with ci settings + job_configs = ci_settings.apply( + job_configs, + pr_info.is_release, + is_pr=pr_info.is_pr, + is_mq=pr_info.is_merge_queue, + labels=pr_info.labels, + ) + + # add all job batches to job's to_do batches + for _job, job_config in job_configs.items(): + batches = [] + for batch in range(job_config.num_batches): + batches.append(batch) + job_config.batches = batches # check jobs in ci cache ci_cache = CiCache.calc_digests_and_create( @@ -1102,6 +1121,7 @@ def main() -> int: pr_info, ci_settings, args.skip_jobs, + args.workflow, ) ci_cache.print_status() @@ -1111,7 +1131,7 @@ def main() -> int: if IS_CI and not pr_info.is_merge_queue: - if pr_info.is_release: + if pr_info.is_release and pr_info.is_push_event: print("Release/master: CI Cache add pending records for all todo jobs") ci_cache.push_pending_all(pr_info.is_release) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a44b15f34c1..a9bdb639835 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -37,11 +37,22 @@ class CI: from ci_utils import GHActions as GHActions from ci_definitions import Labels as Labels from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS + from ci_definitions import WorkFlowNames as WorkFlowNames from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL # Jobs that run for doc related updates _DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] + WORKFLOW_CONFIGS = { + WorkFlowNames.JEPSEN: LabelConfig( + run_jobs=[ + BuildNames.BINARY_RELEASE, + JobNames.JEPSEN_KEEPER, + JobNames.JEPSEN_SERVER, + ] + ) + } # type: Dict[str, LabelConfig] + TAG_CONFIGS = { Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), Tags.CI_SET_ARM: LabelConfig( @@ -68,7 +79,7 @@ class CI: JobNames.STATEFUL_TEST_ASAN, ] ), - } + } # type: Dict[str, LabelConfig] JOB_CONFIGS: Dict[str, JobConfig] = { BuildNames.PACKAGE_RELEASE: CommonJobConfigs.BUILD.with_properties( @@ -599,16 +610,25 @@ class CI: @classmethod def get_workflow_jobs_with_configs( - cls, is_mq: bool, is_docs_only: bool, is_master: bool, is_pr: bool + cls, + is_mq: bool, + is_docs_only: bool, + is_master: bool, + is_pr: bool, + workflow_name: str, ) -> Dict[str, JobConfig]: """ get a list of all jobs for a workflow with configs """ - jobs = [] if is_mq: jobs = MQ_JOBS elif is_docs_only: jobs = cls._DOCS_CHECK_JOBS + elif workflow_name: + assert ( + workflow_name in cls.WORKFLOW_CONFIGS + ), "Workflow name if provided must be configured in WORKFLOW_CONFIGS" + jobs = list(cls.WORKFLOW_CONFIGS[workflow_name].run_jobs) else: # add all jobs jobs = list(cls.JOB_CONFIGS) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index a8d9793f1d3..149177ecba5 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -106,6 +106,14 @@ class Tags(metaclass=WithIter): libFuzzer = "libFuzzer" +class WorkFlowNames(metaclass=WithIter): + """ + CI WorkFlow Names for custom CI runs + """ + + JEPSEN = "JepsenWorkflow" + + class BuildNames(metaclass=WithIter): """ Build' job names diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index c29c5777dba..d6e9765ceb7 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -234,11 +234,4 @@ class CiSettings: for job in add_parents: res[job] = job_configs[job] - for job, job_config in res.items(): - batches = [] - for batch in range(job_config.num_batches): - if not self.job_batches or batch in self.job_batches: - batches.append(batch) - job_config.batches = batches - return res diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index f91a3f080c0..772467d4245 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -9,16 +9,13 @@ from pathlib import Path from typing import Any, List import boto3 # type: ignore -import requests from build_download_helper import ( - download_build_with_progress, read_build_urls, ) from compress_files import compress_fast -from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, S3_URL, TEMP_PATH +from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from get_robot_token import get_parameter_from_ssm -from git_helper import git_runner from pr_info import PRInfo from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults from ssh import SSHKey @@ -32,11 +29,10 @@ KEEPER_DESIRED_INSTANCE_COUNT = 3 SERVER_DESIRED_INSTANCE_COUNT = 4 KEEPER_IMAGE_NAME = "clickhouse/keeper-jepsen-test" -KEEPER_CHECK_NAME = "ClickHouse Keeper Jepsen" +KEEPER_CHECK_NAME = CI.JobNames.JEPSEN_KEEPER SERVER_IMAGE_NAME = "clickhouse/server-jepsen-test" -SERVER_CHECK_NAME = "ClickHouse Server Jepsen" - +SERVER_CHECK_NAME = CI.JobNames.JEPSEN_SERVER SUCCESSFUL_TESTS_ANCHOR = "# Successful tests" INTERMINATE_TESTS_ANCHOR = "# Indeterminate tests" @@ -201,36 +197,14 @@ def main(): # always use latest docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME - if pr_info.is_scheduled or pr_info.is_dispatched: - # get latest clickhouse by the static link for latest master buit - get its version and provide permanent url for this version to the jepsen - build_url = f"{S3_URL}/{S3_BUILDS_BUCKET}/master/amd64/clickhouse" - download_build_with_progress(build_url, Path(TEMP_PATH) / "clickhouse") - git_runner.run(f"chmod +x {TEMP_PATH}/clickhouse") - sha = git_runner.run( - f"{TEMP_PATH}/clickhouse local -q \"select value from system.build_options where name='GIT_HASH'\"" - ) - version_full = git_runner.run( - f'{TEMP_PATH}/clickhouse local -q "select version()"' - ) - version = ".".join(version_full.split(".")[0:2]) - assert len(sha) == 40, f"failed to fetch sha from the binary. result: {sha}" - assert ( - version - ), f"failed to fetch version from the binary. result: {version_full}" - build_url = ( - f"{S3_URL}/{S3_BUILDS_BUCKET}/{version}/{sha}/binary_release/clickhouse" - ) - print(f"Clickhouse version: [{version_full}], sha: [{sha}], url: [{build_url}]") - head = requests.head(build_url, timeout=60) - assert head.status_code == 200, f"Clickhouse binary not found: {build_url}" - else: - build_name = CI.get_required_build_name(check_name) - urls = read_build_urls(build_name, REPORT_PATH) - build_url = None - for url in urls: - if url.endswith("clickhouse"): - build_url = url - assert build_url, "No build url found in the report" + # binary_release assumed to be always ready on the master as it's part of the merge queue workflow + build_name = CI.get_required_build_name(check_name) + urls = read_build_urls(build_name, REPORT_PATH) + build_url = None + for url in urls: + if url.endswith("clickhouse"): + build_url = url + assert build_url, "No build url found in the report" extra_args = "" if args.program == "server": diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 59806a2a8fa..2c8ada7b983 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -337,6 +337,10 @@ class PRInfo: return True return False + @property + def is_push_event(self) -> bool: + return self.event_type == EventType.PUSH + @property def is_scheduled(self) -> bool: return self.event_type == EventType.SCHEDULE diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 44142050821..4a2bd606d0e 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -419,6 +419,32 @@ class TestCIConfig(unittest.TestCase): ] self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + def test_ci_py_for_specific_workflow(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it merge_queue + pr_info.event_type = EventType.SCHEDULE + assert pr_info.number == 0 and not pr_info.is_merge_queue and not pr_info.is_pr + ci_cache = CIPY._configure_jobs( + S3Helper(), + pr_info, + settings, + skip_jobs=False, + dry_run=True, + workflow_name=CI.WorkFlowNames.JEPSEN, + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [ + CI.BuildNames.BINARY_RELEASE, + CI.JobNames.JEPSEN_KEEPER, + CI.JobNames.JEPSEN_SERVER, + ] + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + def test_ci_py_await(self): """ checks ci.py job configuration From fa437b34ec16a0f7fe52f0e4261f964fef2ed606 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jul 2024 03:16:20 +0200 Subject: [PATCH 110/123] Increase timeout for curl in tests --- tests/queries/shell_config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index ef2d89f0218..f7017958635 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -132,7 +132,7 @@ export CLICKHOUSE_URL_INTERSERVER=${CLICKHOUSE_URL_INTERSERVER:="${CLICKHOUSE_PO export CLICKHOUSE_CURL_COMMAND=${CLICKHOUSE_CURL_COMMAND:="curl"} # The queries in CI are prone to sudden delays, and we often don't check for curl # errors, so it makes sense to set a relatively generous timeout. -export CLICKHOUSE_CURL_TIMEOUT=${CLICKHOUSE_CURL_TIMEOUT:="60"} +export CLICKHOUSE_CURL_TIMEOUT=${CLICKHOUSE_CURL_TIMEOUT:="120"} export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="${CLICKHOUSE_CURL_COMMAND} -q -s --max-time ${CLICKHOUSE_CURL_TIMEOUT}"} export CLICKHOUSE_TMP=${CLICKHOUSE_TMP:="."} mkdir -p ${CLICKHOUSE_TMP} From 82959ce5b3e3709f12cb5cf8d50f8ca81858c7ed Mon Sep 17 00:00:00 2001 From: zoomxi <419486879@qq.com> Date: Thu, 25 Jul 2024 09:55:23 +0800 Subject: [PATCH 111/123] format test.py --- tests/integration/test_parallel_replicas_no_replicas/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index 04e3a54e581..62d4b005d94 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -34,7 +34,9 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize("skip_unavailable_shards", [1, 0]) @pytest.mark.parametrize("max_parallel_replicas", [2, 3, 100]) -def test_skip_all_replicas(start_cluster, skip_unavailable_shards, max_parallel_replicas): +def test_skip_all_replicas( + start_cluster, skip_unavailable_shards, max_parallel_replicas +): cluster_name = "test_1_shard_3_unavaliable_replicas" table_name = "tt" create_tables(cluster_name, table_name) From 36e59a1b7083f646b5509e6ef379d1c24f23ad0b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 25 Jul 2024 01:58:27 +0000 Subject: [PATCH 112/123] Final commit hash --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 9b1f47ad8a6..a89d904befe 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 9b1f47ad8a6fcecbeaaead93bd87756ccf658071 +Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 From f239dd67eec9cded889d8aee9e4895da5745541f Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 24 Jul 2024 03:30:04 +0200 Subject: [PATCH 113/123] Fix --- src/Databases/DatabaseReplicated.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 4c079ae5300..706d91d6c9e 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -641,7 +641,10 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, if (is_probably_dropped) return; - ddl_worker = std::make_unique(this, getContext()); + { + std::lock_guard lock{mutex}; + ddl_worker = std::make_unique(this, getContext()); + } ddl_worker->startup(); ddl_worker_initialized = true; }); From aa9908f6da6ee7bf932a649c2884615856b408bb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 24 Jul 2024 04:33:47 +0200 Subject: [PATCH 114/123] Fix harder --- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabaseReplicated.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 706d91d6c9e..dd524e305a1 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -642,7 +642,7 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, return; { - std::lock_guard lock{mutex}; + std::lock_guard lock{ddl_worker_mutex}; ddl_worker = std::make_unique(this, getContext()); } ddl_worker->startup(); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8c3fa7c87f6..9edba0eeb2b 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -149,6 +149,7 @@ private: std::atomic_bool is_recovering = false; std::atomic_bool ddl_worker_initialized = false; std::unique_ptr ddl_worker; + std::mutex ddl_worker_mutex; UInt32 max_log_ptr_at_creation = 0; /// Usually operation with metadata are single-threaded because of the way replication works, From cea3bbc45b530742ea5d0374093278494ec5a9b8 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 24 Jul 2024 07:06:08 +0200 Subject: [PATCH 115/123] More checks --- src/Databases/DatabaseReplicated.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index dd524e305a1..97a9b65f6b4 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -559,7 +559,7 @@ void DatabaseReplicated::createEmptyLogEntry(const ZooKeeperPtr & current_zookee bool DatabaseReplicated::waitForReplicaToProcessAllEntries(UInt64 timeout_ms) { - if (!ddl_worker || is_probably_dropped) + if (!ddl_worker_initialized || !ddl_worker || is_probably_dropped) return false; return ddl_worker->waitForReplicaToProcessAllEntries(timeout_ms); } @@ -686,7 +686,7 @@ bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool LOG_TEST(log, "Current in-memory metadata digest: {}", tables_metadata_digest); /// Database is probably being dropped - if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) + if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker_initialized || !ddl_worker || !ddl_worker->isCurrentlyActive())) return true; UInt64 local_digest = 0; @@ -1414,7 +1414,7 @@ void DatabaseReplicated::renameDatabase(ContextPtr query_context, const String & void DatabaseReplicated::stopReplication() { - if (ddl_worker) + if (ddl_worker_initialized && ddl_worker) ddl_worker->shutdown(); } From 80b9b13771bb08a1d8d713aabd55ac0471292b1a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 24 Jul 2024 07:15:35 +0200 Subject: [PATCH 116/123] Less checks, more locks --- src/Databases/DatabaseReplicated.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 97a9b65f6b4..c90f4ea4719 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -559,8 +559,11 @@ void DatabaseReplicated::createEmptyLogEntry(const ZooKeeperPtr & current_zookee bool DatabaseReplicated::waitForReplicaToProcessAllEntries(UInt64 timeout_ms) { - if (!ddl_worker_initialized || !ddl_worker || is_probably_dropped) - return false; + { + std::lock_guard lock{ddl_worker_mutex}; + if (!ddl_worker || is_probably_dropped) + return false; + } return ddl_worker->waitForReplicaToProcessAllEntries(timeout_ms); } @@ -686,8 +689,11 @@ bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool LOG_TEST(log, "Current in-memory metadata digest: {}", tables_metadata_digest); /// Database is probably being dropped - if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker_initialized || !ddl_worker || !ddl_worker->isCurrentlyActive())) - return true; + { + std::lock_guard lock{ddl_worker_mutex}; + if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) + return true; + } UInt64 local_digest = 0; { @@ -1414,7 +1420,8 @@ void DatabaseReplicated::renameDatabase(ContextPtr query_context, const String & void DatabaseReplicated::stopReplication() { - if (ddl_worker_initialized && ddl_worker) + std::lock_guard lock{ddl_worker_mutex}; + if (ddl_worker) ddl_worker->shutdown(); } From e21d23d04daf2577db63e26cee22bc686e10833e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 24 Jul 2024 07:55:36 +0200 Subject: [PATCH 117/123] Less locks --- src/Databases/DatabaseReplicated.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c90f4ea4719..c737ece02ec 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -689,11 +689,8 @@ bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool LOG_TEST(log, "Current in-memory metadata digest: {}", tables_metadata_digest); /// Database is probably being dropped - { - std::lock_guard lock{ddl_worker_mutex}; - if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) - return true; - } + if (!local_context->getZooKeeperMetadataTransaction() && (!ddl_worker || !ddl_worker->isCurrentlyActive())) + return true; UInt64 local_digest = 0; { From 4ee409094d4b1b9a8c8f4da5f1cf55d78a43b11f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Jul 2024 12:12:23 +0200 Subject: [PATCH 118/123] Update tests/performance/decimal_aggregates.xml --- tests/performance/decimal_aggregates.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/decimal_aggregates.xml b/tests/performance/decimal_aggregates.xml index b204dddea6e..724d0c5d0e6 100644 --- a/tests/performance/decimal_aggregates.xml +++ b/tests/performance/decimal_aggregates.xml @@ -9,6 +9,7 @@ INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000, 100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(200000000, 100000000) SETTINGS max_threads = 2 INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(300000000, 100000000) SETTINGS max_threads = 2 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(400000000, 100000000) SETTINGS max_threads = 2 DROP TABLE IF EXISTS t From 93a2dbf85c446cd5efa802dfe8fc99e2a82f394f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 Jul 2024 17:02:11 +0200 Subject: [PATCH 119/123] Move syntax part on top --- docs/en/sql-reference/statements/system.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index f4780fd41c1..35f2f15dd80 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -28,11 +28,14 @@ SYSTEM RELOAD DICTIONARIES [ON CLUSTER cluster_name] Completely reloads a dictionary `dictionary_name`, regardless of the state of the dictionary (LOADED / NOT_LOADED / FAILED). Always returns `Ok.` regardless of the result of updating the dictionary. -The status of the dictionary can be checked by querying the `system.dictionaries` table. ``` sql SYSTEM RELOAD DICTIONARY [ON CLUSTER cluster_name] dictionary_name +``` +The status of the dictionary can be checked by querying the `system.dictionaries` table. + +``` sql SELECT name, status FROM system.dictionaries; ``` From ee193ffa019fc0f6104c1c5010ba00cd1993c8bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 19:25:58 +0200 Subject: [PATCH 120/123] Lower max allocation size in query fuzzer --- docker/test/fuzzer/query-fuzzer-tweaks-users.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index e2a4976b385..d5b876a4c85 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -20,7 +20,7 @@ - 10G + 5G From e46512a3bed4cd260042acbde4fcbef5cb83e032 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 25 Jul 2024 16:59:10 +0000 Subject: [PATCH 121/123] Try fix 02481_async_insert_race_long https://github.com/ClickHouse/clickhouse-private/issues/13101 My assumption is that for high --insert_keeper_fault_injection_probability values, entries don't leave the sink, causing the queue memory consumption to grow, eventually hitting the memory limit and the MEMORY_LIMIT_EXCEEDED error. Later, we may raise the insert_keeper_fault_injection_probability to a small positive value. --- tests/queries/0_stateless/02481_async_insert_race_long.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_race_long.sh b/tests/queries/0_stateless/02481_async_insert_race_long.sh index d8153967e9a..b0088017d32 100755 --- a/tests/queries/0_stateless/02481_async_insert_race_long.sh +++ b/tests/queries/0_stateless/02481_async_insert_race_long.sh @@ -13,7 +13,7 @@ function insert1() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --wait_for_async_insert 0 -q 'INSERT INTO async_inserts_race FORMAT CSV 1,"a"' + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 0 -q 'INSERT INTO async_inserts_race FORMAT CSV 1,"a"' done } @@ -21,7 +21,7 @@ function insert2() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --wait_for_async_insert 0 -q 'INSERT INTO async_inserts_race FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 0 -q 'INSERT INTO async_inserts_race FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' done } @@ -29,7 +29,7 @@ function insert3() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" & + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" & sleep 0.05 done From 7b8c41818bcec1c567c85b15d916fd2e064b7482 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 25 Jul 2024 19:34:41 +0000 Subject: [PATCH 122/123] Uncomment accidentally commented out code in QueryProfiler --- src/Common/QueryProfiler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index a7717a4288a..f6524088102 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -57,7 +57,7 @@ namespace auto saved_errno = errno; /// We must restore previous value of errno in signal handler. -#if defined(OS_LINUX) && false //asdqwe +#if defined(OS_LINUX) if (info) { int overrun_count = info->si_overrun; @@ -92,7 +92,7 @@ namespace constexpr bool sanitizer = false; #endif - //asdqwe asynchronous_stack_unwinding = true; + asynchronous_stack_unwinding = true; if (sanitizer || 0 == sigsetjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1)) { stack_trace.emplace(signal_context); From 992a2764855d61ea8c8a75c727e6e0fd33054c1a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 25 Jul 2024 20:23:45 +0000 Subject: [PATCH 123/123] uncomment also write_trace_iteration --- src/Common/QueryProfiler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index f6524088102..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -33,7 +33,7 @@ namespace DB namespace { #if defined(OS_LINUX) - //thread_local size_t write_trace_iteration = 0; + thread_local size_t write_trace_iteration = 0; #endif /// Even after timer_delete() the signal can be delivered, /// since it does not do anything with pending signals.