From f0fdeb6d3ef9ae51098d11a0a22bb138b6a58df0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 24 Feb 2024 22:36:29 +0100 Subject: [PATCH 001/321] 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/321] 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/321] 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/321] 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/321] 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/321] 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/321] 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 38537a00aa9e7185b69c066cd6809c54487ecf4e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 18:16:12 +0000 Subject: [PATCH 008/321] Make ActionsDAGPtr unique_ptr. --- src/Core/InterpolateDescription.cpp | 2 +- src/Core/InterpolateDescription.h | 2 +- src/Functions/indexHint.h | 4 +- src/Interpreters/ActionsDAG.cpp | 46 ++++++++------- src/Interpreters/ActionsDAG.h | 11 ++-- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ActionsVisitor.h | 2 +- src/Interpreters/ExpressionActions.cpp | 20 ++++++- src/Interpreters/ExpressionActions.h | 6 +- src/Interpreters/ExpressionAnalyzer.cpp | 27 ++++----- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 14 ++--- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/TableJoin.cpp | 4 +- src/Interpreters/WindowDescription.h | 6 +- src/Interpreters/addMissingDefaults.cpp | 2 +- src/Interpreters/addMissingDefaults.h | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 2 +- src/Interpreters/inplaceBlockConversions.h | 2 +- src/Planner/CollectTableExpressionData.cpp | 6 +- src/Planner/Planner.cpp | 49 ++++++++-------- src/Planner/PlannerActionsVisitor.cpp | 6 +- src/Planner/PlannerJoinTree.cpp | 58 ++++++++++--------- src/Planner/PlannerJoinTree.h | 2 +- src/Planner/PlannerJoins.cpp | 12 ++-- src/Planner/PlannerWindowFunctions.cpp | 13 ++++- src/Planner/PlannerWindowFunctions.h | 2 +- src/Planner/Utils.cpp | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 4 +- src/Processors/QueryPlan/CubeStep.cpp | 4 +- src/Processors/QueryPlan/ExpressionStep.cpp | 12 ++-- src/Processors/QueryPlan/ExpressionStep.h | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 11 ++-- src/Processors/QueryPlan/FilterStep.h | 2 +- .../Optimizations/distinctReadInOrder.cpp | 12 ++-- .../Optimizations/filterPushDown.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizePrimaryKeyCondition.cpp | 6 +- .../Optimizations/optimizeReadInOrder.cpp | 4 +- .../optimizeUseAggregateProjection.cpp | 4 +- .../optimizeUseNormalProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 4 +- .../Optimizations/projectionsCommon.h | 2 +- .../Optimizations/removeRedundantDistinct.cpp | 23 ++++---- .../QueryPlan/ReadFromMergeTree.cpp | 38 ++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- .../QueryPlan/SourceStepWithFilter.cpp | 8 +-- .../QueryPlan/SourceStepWithFilter.h | 3 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 8 +-- src/Processors/QueryPlan/TotalsHavingStep.h | 2 +- src/Processors/QueryPlan/WindowStep.h | 2 +- src/Processors/SourceWithKeyCondition.h | 4 +- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 8 +-- src/Storages/KeyDescription.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 4 +- src/Storages/MergeTree/KeyCondition.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 +-- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +-- .../MergeTree/MergeTreeDataSelectExecutor.h | 4 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 4 +- .../MergeTree/MergeTreeIndexBloomFilter.h | 4 +- .../MergeTreeIndexBloomFilterText.cpp | 4 +- .../MergeTree/MergeTreeIndexBloomFilterText.h | 4 +- .../MergeTree/MergeTreeIndexFullText.cpp | 4 +- .../MergeTree/MergeTreeIndexFullText.h | 4 +- .../MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- .../MergeTree/MergeTreeIndexHypothesis.h | 2 +- .../MergeTree/MergeTreeIndexMinMax.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndexMinMax.h | 4 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndexSet.h | 4 +- .../MergeTree/MergeTreeIndexUSearch.cpp | 2 +- .../MergeTree/MergeTreeIndexUSearch.h | 2 +- src/Storages/MergeTree/MergeTreeIndices.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeSequentialSource.cpp | 4 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 32 +++++----- src/Storages/MergeTree/PartitionPruner.cpp | 2 +- src/Storages/MergeTree/PartitionPruner.h | 2 +- .../StorageObjectStorageSource.cpp | 2 +- src/Storages/SelectQueryInfo.h | 4 +- src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageMerge.cpp | 10 ++-- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.cpp | 4 +- src/Storages/StorageView.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 4 +- src/Storages/VirtualColumnUtils.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 12 ++-- 98 files changed, 355 insertions(+), 318 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index d828c2e85e9..76bbefdcfd7 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -14,7 +14,7 @@ namespace DB { InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) - : actions(actions_) + : actions(std::move(actions_)) { for (const auto & name_type : actions->getRequiredColumns()) { diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 62d7120508b..73579aebee4 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -11,7 +11,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; using Aliases = std::unordered_map; /// Interpolate description diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 3b71c7a5585..8fd7b751760 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -2,14 +2,12 @@ #include #include #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; - /** The `indexHint` function takes any number of any arguments and always returns one. * * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 34f3e0a98bd..23e1e5ee152 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -626,7 +626,7 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { - auto actions = std::make_shared(); + auto actions = std::make_unique(); std::unordered_map copy_map; struct Frame @@ -1248,25 +1248,29 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) ActionsDAGPtr ActionsDAG::clone() const { - auto actions = std::make_shared(); + std::unordered_map old_to_new_nodes; + return clone(old_to_new_nodes); +} - std::unordered_map copy_map; +ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const +{ + auto actions = std::make_unique(); for (const auto & node : nodes) { auto & copy_node = actions->nodes.emplace_back(node); - copy_map[&node] = ©_node; + old_to_new_nodes[&node] = ©_node; } for (auto & node : actions->nodes) for (auto & child : node.children) - child = copy_map[child]; + child = old_to_new_nodes[child]; for (const auto & output_node : outputs) - actions->outputs.push_back(copy_map[output_node]); + actions->outputs.push_back(old_to_new_nodes[output_node]); for (const auto & input_node : inputs) - actions->inputs.push_back(copy_map[input_node]); + actions->inputs.push_back(old_to_new_nodes[input_node]); return actions; } @@ -1421,7 +1425,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (add_casted_columns && mode != MatchColumnsMode::Name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); - auto actions_dag = std::make_shared(source); + auto actions_dag = std::make_unique(source); NodeRawConstPtrs projection(num_result_columns); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1549,7 +1553,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) { - auto adding_column_action = std::make_shared(); + auto adding_column_action = std::make_unique(); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1570,7 +1574,7 @@ ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) /// Some actions could become unused. Do not drop inputs to preserve the header. first.removeUnusedActions(false); - return std::make_shared(std::move(first)); + return std::make_unique(std::move(first)); } void ActionsDAG::mergeInplace(ActionsDAG && second) @@ -1963,12 +1967,12 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split second_inputs.push_back(cur.to_second); } - auto first_actions = std::make_shared(); + auto first_actions = std::make_unique(); first_actions->nodes.swap(first_nodes); first_actions->outputs.swap(first_outputs); first_actions->inputs.swap(first_inputs); - auto second_actions = std::make_shared(); + auto second_actions = std::make_unique(); second_actions->nodes.swap(second_nodes); second_actions->outputs.swap(second_outputs); second_actions->inputs.swap(second_inputs); @@ -2302,7 +2306,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti if (conjunction.empty()) return nullptr; - auto actions = std::make_shared(); + auto actions = std::make_unique(); FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); @@ -2866,7 +2870,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( bool visited_children = false; }; - auto result_dag = std::make_shared(); + auto result_dag = std::make_unique(); std::unordered_map result_inputs; std::unordered_map node_to_result_node; @@ -2964,7 +2968,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( const auto & index_hint_args = index_hint->getActions()->getOutputs(); if (index_hint_args.empty()) - index_hint_filter_dag = std::make_shared(); + index_hint_filter_dag = std::make_unique(); else index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, node_name_to_input_node_column, @@ -3108,10 +3112,10 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs( return nodes; } -FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_) - :actions(actions_) +FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_) + //: actions(actions_) { - const auto & actions_outputs = actions->getOutputs(); + const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) { /// find input node which refers to the output node @@ -3147,10 +3151,10 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp return it->second; } -FindAliasForInputName::FindAliasForInputName(const ActionsDAGPtr & actions_) - :actions(actions_) +FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_) + //: actions(actions_) { - const auto & actions_outputs = actions->getOutputs(); + const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) { /// find input node which corresponds to alias diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index c9974fd849c..4a840885b6a 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -12,7 +12,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class IExecutableFunction; using ExecutableFunctionPtr = std::shared_ptr; @@ -262,6 +262,7 @@ public: #endif ActionsDAGPtr clone() const; + ActionsDAGPtr clone(std::unordered_map & old_to_new_nodes) const; static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); @@ -480,11 +481,11 @@ class FindOriginalNodeForOutputName using NameToNodeIndex = std::unordered_map; public: - explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions); + explicit FindOriginalNodeForOutputName(const ActionsDAG & actions); const ActionsDAG::Node * find(const String & output_name); private: - ActionsDAGPtr actions; + //const ActionsDAG & actions; NameToNodeIndex index; }; @@ -493,11 +494,11 @@ class FindAliasForInputName using NameToNodeIndex = std::unordered_map; public: - explicit FindAliasForInputName(const ActionsDAGPtr & actions); + explicit FindAliasForInputName(const ActionsDAG & actions); const ActionsDAG::Node * find(const String & name); private: - ActionsDAGPtr actions; + //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9e56d740e5e..1838a7b04b9 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1009,7 +1009,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & dag.project(args); auto index_hint = std::make_shared(); - index_hint->setActions(std::make_shared(std::move(dag))); + index_hint->setActions(std::make_unique(std::move(dag))); // Arguments are removed. We add function instead of constant column to avoid constant folding. data.addFunction(std::make_unique(index_hint), {}, column_name); @@ -1272,7 +1272,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - std::make_shared(std::move(lambda_dag)), + std::make_unique(std::move(lambda_dag)), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 46d2d60e461..496d9b9b587 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -22,7 +22,7 @@ class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 7f96c927d82..7cbf5afd763 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -73,7 +73,25 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio ExpressionActionsPtr ExpressionActions::clone() const { - return std::make_shared(*this); + auto copy = std::make_shared(ExpressionActions()); + + std::unordered_map copy_map; + copy->actions_dag = actions_dag->clone(copy_map); + copy->actions = actions; + for (auto & action : copy->actions) + action.node = copy_map[action.node]; + + copy->num_columns = num_columns; + + copy->required_columns = required_columns; + copy->input_positions = input_positions; + copy->result_positions = result_positions; + copy->sample_block = sample_block; + + copy->project_inputs = project_inputs; + copy->settings = settings; + + return copy; } namespace diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index ddffe022215..63ea989bd5e 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -84,10 +84,9 @@ private: ExpressionActionsSettings settings; public: - ExpressionActions() = delete; explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); - ExpressionActions(const ExpressionActions &) = default; - ExpressionActions & operator=(const ExpressionActions &) = default; + ExpressionActions(ExpressionActions &&) = default; + ExpressionActions & operator=(ExpressionActions &&) = default; const Actions & getActions() const { return actions; } const std::list & getNodes() const { return actions_dag->getNodes(); } @@ -131,6 +130,7 @@ public: ExpressionActionsPtr clone() const; private: + ExpressionActions() = default; void checkLimits(const ColumnsWithTypeAndName & columns) const; void linearizeActions(const std::unordered_set & lazy_executed_nodes); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 62cddd9caf7..be00e37c751 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -658,7 +658,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, with_alias->getColumnName(), 1 /* direction */, 1 /* nulls_direction */)); - auto actions_dag = std::make_shared(aggregated_columns); + auto actions_dag = std::make_unique(aggregated_columns); getRootActions(column_ast, false, *actions_dag); desc.partition_by_actions.push_back(std::move(actions_dag)); } @@ -679,7 +679,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, order_by_element.direction, order_by_element.nulls_direction)); - auto actions_dag = std::make_shared(aggregated_columns); + auto actions_dag = std::make_unique(aggregated_columns); getRootActions(column_ast, false, *actions_dag); desc.order_by_actions.push_back(std::move(actions_dag)); } @@ -823,13 +823,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) makeWindowDescriptionFromAST(*current_context, window_descriptions, desc, &definition); + auto full_sort_description = desc.full_sort_description; + auto [it, inserted] = window_descriptions.insert( - {default_window_name, desc}); + {default_window_name, std::move(desc)}); if (!inserted) { - assert(it->second.full_sort_description - == desc.full_sort_description); + assert(it->second.full_sort_description == full_sort_description); } it->second.window_functions.push_back(window_function); @@ -1353,10 +1354,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_unique(columns_after_join); getRootActions(child, only_types, *actions_dag); group_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); + std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } } @@ -1606,10 +1607,10 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr { for (const auto & child : select_query->orderBy()->children) { - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_unique(columns_after_join); getRootActions(child, only_types, *actions_dag); order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); + std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } } @@ -1799,7 +1800,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) { - auto actions = std::make_shared(constant_inputs); + auto actions = std::make_unique(constant_inputs); getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); return actions; } @@ -1807,7 +1808,7 @@ ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndNam ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs) { auto actions = getConstActionsDAG(constant_inputs); - return std::make_shared(actions, ExpressionActionsSettings::fromContext(getContext())); + return std::make_shared(std::move(actions), ExpressionActionsSettings::fromContext(getContext())); } std::unique_ptr SelectQueryExpressionAnalyzer::getJoinedPlan() @@ -1878,7 +1879,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (prewhere_dag_and_flags) { - auto dag = std::make_shared(std::move(prewhere_dag_and_flags->dag)); + auto dag = std::make_unique(std::move(prewhere_dag_and_flags->dag)); prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); prewhere_dag_and_flags.reset(); } @@ -1945,7 +1946,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { auto dag = prewhere_dag_and_flags->dag.clone(); ExpressionActions( - dag, + std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 12d6dce8f72..e44a5891e77 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -39,7 +39,7 @@ class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 927bafe4bfb..4dbdebd0d06 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -384,7 +384,7 @@ Chain InterpreterInsertQuery::buildPreSinkChain( context_ptr, null_as_default); - auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + auto adding_missing_defaults_actions = std::make_shared(std::move(adding_missing_defaults_dag)); /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) @@ -597,7 +597,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + auto actions = std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f5b54ec64cb..64a17a7ba87 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1302,7 +1302,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( result_columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); - interpolate_descr = std::make_shared(merge_dag, aliases); + interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } return interpolate_descr; @@ -2042,7 +2042,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(prewhere_info.row_level_filter), + std::make_shared(prewhere_info.row_level_filter->clone()), prewhere_info.row_level_column_name, true); }); } @@ -2050,7 +2050,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(prewhere_info.prewhere_actions), + header, std::make_shared(prewhere_info.prewhere_actions->clone()), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2094,8 +2094,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. - analysis.prewhere_info = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); - analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; + analysis.prewhere_info = std::make_shared(std::move(analysis.filter_info->actions), analysis.filter_info->column_name); + analysis.prewhere_info->remove_prewhere_column = std::move(analysis.filter_info->do_remove_column); analysis.prewhere_info->need_filter = true; analysis.filter_info = nullptr; } @@ -2103,8 +2103,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; - analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name; + analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..1bb770bf561 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1211,7 +1211,7 @@ void MutationsInterpreter::Source::read( MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, part, required_columns, - apply_deleted_mask_, filter, context_, + apply_deleted_mask_, std::move(filter), context_, getLogger("MutationsInterpreter")); } else diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 6191eb73fd4..baf3a743f40 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -470,7 +470,7 @@ static ActionsDAGPtr createWrapWithTupleActions( if (column_names_to_wrap.empty()) return nullptr; - auto actions_dag = std::make_shared(source_columns); + auto actions_dag = std::make_unique(source_columns); FunctionOverloadResolverPtr func_builder = std::make_unique(std::make_shared()); @@ -616,7 +616,7 @@ TableJoin::createConvertingActions( mergeDags(right_dag, std::move(new_right_dag)); } - return {left_dag, right_dag}; + return {std::move(left_dag), std::move(right_dag)}; } template diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index c26e4517c9a..17bfe619c30 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -14,7 +14,7 @@ namespace DB class ASTFunction; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; struct WindowFunctionDescription { @@ -93,8 +93,8 @@ struct WindowDescription // then by ORDER BY. This field holds this combined sort order. SortDescription full_sort_description; - std::vector partition_by_actions; - std::vector order_by_actions; + std::vector> partition_by_actions; + std::vector> order_by_actions; WindowFrame frame; diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index fbf17d7efb7..929999c8c37 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -21,7 +21,7 @@ ActionsDAGPtr addMissingDefaults( ContextPtr context, bool null_as_default) { - auto actions = std::make_shared(header.getColumnsWithTypeAndName()); + auto actions = std::make_unique(header.getColumnsWithTypeAndName()); auto & index = actions->getOutputs(); /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 0a3d4de478c..94afd806dfd 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -15,7 +15,7 @@ class NamesAndTypesList; class ColumnsDescription; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /** Adds three types of columns into block * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 239cce5b427..b000264ae33 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -163,7 +163,7 @@ ActionsDAGPtr createExpressions( auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context}; - auto dag = std::make_shared(header.getNamesAndTypesList()); + auto dag = std::make_unique(header.getNamesAndTypesList()); auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns); dag = ActionsDAG::merge(std::move(*dag), std::move(*actions)); diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index bea44bf6db9..ffc77561e79 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -24,7 +24,7 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Create actions which adds missing defaults to block according to required_columns using columns description /// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index d5e39a9f123..162d3fe8d11 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -88,7 +88,7 @@ public: auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); - ActionsDAGPtr alias_column_actions_dag = std::make_shared(); + ActionsDAGPtr alias_column_actions_dag = std::make_unique(); PlannerActionsVisitor actions_visitor(planner_context, false); auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression()); if (outputs.size() != 1) @@ -97,7 +97,7 @@ public: const auto & column_name = column_node->getColumnName(); const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name); alias_column_actions_dag->addOrReplaceInOutputs(alias_node); - table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns); + table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns); } return; @@ -335,7 +335,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr collect_source_columns_visitor.setKeepAliasColumns(false); collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); - auto prewhere_actions_dag = std::make_shared(); + auto prewhere_actions_dag = std::make_unique(); QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2d42ed73223..681ae7e6ac4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -213,7 +213,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & if (!read_from_dummy) continue; - auto filter_actions = read_from_dummy->getFilterActionsDAG(); + auto filter_actions = read_from_dummy->detachFilterActionsDAG(); const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; } @@ -331,13 +331,13 @@ public: void addExpressionStep(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { auto actions = expression_actions->dag.clone(); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); @@ -346,13 +346,13 @@ void addExpressionStep(QueryPlan & query_plan, void addFilterStep(QueryPlan & query_plan, const FilterAnalysisResult & filter_analysis_result, const std::string & step_description, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { auto actions = filter_analysis_result.filter_actions->dag.clone(); if (filter_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), actions, filter_analysis_result.filter_column_name, @@ -544,7 +544,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -560,7 +560,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); } auto totals_having_step = std::make_unique( @@ -714,7 +714,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (query_node.hasInterpolate()) { - auto interpolate_actions_dag = std::make_shared(); + auto interpolate_actions_dag = std::make_unique(); auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) { @@ -885,7 +885,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, const QueryTreeNodePtr & query_tree, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { const auto & query_node = query_tree->as(); @@ -932,14 +932,14 @@ void addWindowSteps(QueryPlan & query_plan, const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - auto window_descriptions = window_analysis_result.window_descriptions; - sortWindowDescriptions(window_descriptions); + const auto & window_descriptions = window_analysis_result.window_descriptions; + auto perm = sortWindowDescriptions(window_descriptions); size_t window_descriptions_size = window_descriptions.size(); for (size_t i = 0; i < window_descriptions_size; ++i) { - const auto & window_description = window_descriptions[i]; + const auto & window_description = window_descriptions[perm[i]]; /** We don't need to sort again if the input from previous window already * has suitable sorting. Also don't create sort steps when there are no @@ -952,8 +952,9 @@ void addWindowSteps(QueryPlan & query_plan, bool need_sort = !window_description.full_sort_description.empty(); if (need_sort && i != 0) { - need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description) - || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[i - 1].partition_by.size()); + auto prev = perm[i - 1]; + need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[prev].full_sort_description) + || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[prev].partition_by.size()); } if (need_sort) { @@ -1054,9 +1055,9 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana } } -void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set & useful_sets) +void collectSetsFromActionsDAG(const ActionsDAG & dag, std::unordered_set & useful_sets) { - for (const auto & node : dag->getNodes()) + for (const auto & node : dag.getNodes()) { if (node.column) { @@ -1075,7 +1076,7 @@ void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set(adaptor->getFunction().get())) { - collectSetsFromActionsDAG(index_hint->getActions(), useful_sets); + collectSetsFromActionsDAG(*index_hint->getActions(), useful_sets); } } } @@ -1086,13 +1087,13 @@ void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) + const std::vector & result_actions_to_execute) { auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; - for (const auto & actions_to_execute : result_actions_to_execute) - collectSetsFromActionsDAG(actions_to_execute, useful_sets); + for (const auto * actions_to_execute : result_actions_to_execute) + collectSetsFromActionsDAG(*actions_to_execute, useful_sets); auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); @@ -1448,7 +1449,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(filters.filter_actions); + table_expression_data.setFilterActions(filters.filter_actions->clone()); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } @@ -1539,15 +1540,15 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info); - std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); + std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) { if (table_expression_data.getPrewhereFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions()); + result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions().get()); if (table_expression_data.getRowLevelFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions()); + result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions().get()); } if (query_processing_info.isIntermediateStage()) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7a12d5d690d..59ec7778e21 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -757,7 +757,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } - auto lambda_actions_dag = std::make_shared(); + auto lambda_actions_dag = std::make_unique(); actions_stack.emplace_back(*lambda_actions_dag, node); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); @@ -765,7 +765,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); - auto lambda_actions = std::make_shared(lambda_actions_dag, expression_actions_settings); + auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); Names captured_column_names; ActionsDAG::NodeRawConstPtrs lambda_children; @@ -879,7 +879,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & function_node = node->as(); auto function_node_name = action_node_name_helper.calculateActionNodeName(node); - auto index_hint_actions_dag = std::make_shared(); + auto index_hint_actions_dag = std::make_unique(); auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); std::unordered_set index_hint_actions_dag_output_node_names; PlannerActionsVisitor actions_visitor(planner_context); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 43b223172e6..918cfad703e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,7 +591,7 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) std::unique_ptr createComputeAliasColumnsStep( const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { - ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared(current_data_stream.header.getColumnsWithTypeAndName()); + ActionsDAGPtr merged_alias_columns_actions_dag = std::make_unique(current_data_stream.header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) @@ -646,7 +646,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions(); + table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions()->clone(); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = prewhere_actions; + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -787,7 +787,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres const auto & columns_names = table_expression_data.getColumnNames(); std::vector> where_filters; - const auto add_filter = [&](const FilterDAGInfo & filter_info, std::string description) + const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description) { if (!filter_info.actions) return; @@ -805,34 +805,34 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = filter_info.actions; + prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = filter_info.actions; + prewhere_info->row_level_filter = std::move(filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } else { - where_filters.emplace_back(filter_info, std::move(description)); + where_filters.emplace_back(std::move(filter_info), std::move(description)); } } else { - where_filters.emplace_back(filter_info, std::move(description)); + where_filters.emplace_back(std::move(filter_info), std::move(description)); } }; auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); - add_filter(row_policy_filter_info, "Row-level security filter"); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions); + table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions->clone()); + add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { @@ -1063,7 +1063,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (from_stage == QueryProcessingStage::FetchColumns) { - auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto rename_actions_dag = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; for (auto & output_node : rename_actions_dag->getOutputs()) @@ -1077,7 +1077,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); - auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); rename_step->setStepDescription("Change column names to column identifiers"); query_plan.addStep(std::move(rename_step)); } @@ -1117,7 +1117,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); for (auto & output_node : cast_actions_dag->getOutputs()) { @@ -1178,6 +1178,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); + left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); + left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); + join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); @@ -1223,7 +1226,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); for (auto & output_node : cast_actions_dag->getOutputs()) { @@ -1381,9 +1384,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_clauses_and_actions.mixed_join_expressions_actions) { + left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( - join_clauses_and_actions.mixed_join_expressions_actions, + std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); } } @@ -1537,7 +1541,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto drop_unused_columns_after_join_actions_dag = std::make_unique(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; std::optional first_skipped_column_node_index; @@ -1582,14 +1586,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` - for (auto && action_dag : right_join_tree_query_plan.actions_dags) + for (const auto * action_dag : right_join_tree_query_plan.actions_dags) left_join_tree_query_plan.actions_dags.emplace_back(action_dag); - if (join_clauses_and_actions.left_join_expressions_actions) - left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions)); - if (join_clauses_and_actions.right_join_expressions_actions) - left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions)); - if (join_clauses_and_actions.mixed_join_expressions_actions) - left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions); + // if (join_clauses_and_actions.left_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); + // if (join_clauses_and_actions.right_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); + // if (join_clauses_and_actions.mixed_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping); auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping; @@ -1619,7 +1623,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto plan = std::move(join_tree_query_plan.query_plan); auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr array_join_action_dag = std::make_shared(plan_output_columns); + ActionsDAGPtr array_join_action_dag = std::make_unique(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); std::unordered_set array_join_expressions_output_nodes; @@ -1642,13 +1646,13 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); - join_tree_query_plan.actions_dags.push_back(array_join_action_dag); + join_tree_query_plan.actions_dags.push_back(array_join_action_dag.get()); - auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), array_join_action_dag); + auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); plan.addStep(std::move(array_join_actions)); - auto drop_unused_columns_before_array_join_actions_dag = std::make_shared(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_before_array_join_actions_dag_updated_outputs_names; diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index 9110b2bfef9..675079427eb 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -16,7 +16,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - std::vector actions_dags{}; + std::vector actions_dags{}; std::unordered_map query_node_to_plan_step_mapping{}; }; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 84efdd21336..45842c0d705 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -376,8 +376,8 @@ JoinClausesAndActions buildJoinClausesAndActions( const JoinNode & join_node, const PlannerContextPtr & planner_context) { - ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); - ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); + ActionsDAGPtr left_join_actions = std::make_unique(left_table_expression_columns); + ActionsDAGPtr right_join_actions = std::make_unique(right_table_expression_columns); ColumnsWithTypeAndName mixed_table_expression_columns; for (const auto & left_column : left_table_expression_columns) { @@ -387,7 +387,7 @@ JoinClausesAndActions buildJoinClausesAndActions( { mixed_table_expression_columns.push_back(right_column); } - ActionsDAGPtr mixed_join_actions = std::make_shared(mixed_table_expression_columns); + ActionsDAGPtr mixed_join_actions = std::make_unique(mixed_table_expression_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -601,7 +601,7 @@ JoinClausesAndActions buildJoinClausesAndActions( /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - auto mixed_join_expressions_actions = std::make_shared(mixed_table_expression_columns); + auto mixed_join_expressions_actions = std::make_unique(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) @@ -611,14 +611,14 @@ JoinClausesAndActions buildJoinClausesAndActions( mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; mixed_join_expressions_actions->removeUnusedActions(required_names); - result.mixed_join_expressions_actions = mixed_join_expressions_actions; + result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } else { const auto & join_clause = result.join_clauses.front(); const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes(); auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); - result.mixed_join_expressions_actions = mixed_join_expressions_actions; + result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } auto outputs = result.mixed_join_expressions_actions->getOutputs(); if (outputs.size() != 1) diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index ce74d82c08d..9deceeef9a3 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -120,7 +120,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & return result; } -void sortWindowDescriptions(std::vector & window_descriptions) +std::vector sortWindowDescriptions(const std::vector & window_descriptions) { auto window_description_comparator = [](const WindowDescription & lhs, const WindowDescription & rhs) { @@ -151,7 +151,16 @@ void sortWindowDescriptions(std::vector & window_descriptions return left.size() > right.size(); }; - ::sort(window_descriptions.begin(), window_descriptions.end(), window_description_comparator); + auto comparator = [&](size_t lhs, size_t rhs) + { + return window_description_comparator(window_descriptions[lhs], window_descriptions[rhs]); + }; + + std::vector perm(window_descriptions.size()); + std::iota(perm.begin(), perm.end(), 0U); + ::sort(perm.begin(), perm.end(), comparator); + + return perm; } } diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h index 1552ef5a71f..3039ecefc4b 100644 --- a/src/Planner/PlannerWindowFunctions.h +++ b/src/Planner/PlannerWindowFunctions.h @@ -15,6 +15,6 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & /** Try to sort window descriptions in such an order that the window with the longest * sort description goes first, and all window that use its prefixes follow. */ -void sortWindowDescriptions(std::vector & window_descriptions); +std::vector sortWindowDescriptions(const std::vector & window_descriptions); } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 18a6d297838..493ecf5ef53 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -440,7 +440,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSets(filter_query_tree, *planner_context); - auto filter_actions_dag = std::make_shared(); + auto filter_actions_dag = std::make_unique(); PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 0d7e05af1de..64ba7f7cd2a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -303,7 +303,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B const auto & header = ports[set_counter]->getHeader(); /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - auto dag = std::make_shared(header.getColumnsWithTypeAndName()); + auto dag = std::make_unique(header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(output_header.columns() + 1); @@ -347,7 +347,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B } dag->getOutputs().swap(outputs); - auto expression = std::make_shared(dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); connect(*ports[set_counter], transform->getInputPort()); diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index d010a3327a6..b6c70061987 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -36,7 +36,7 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { - auto dag = std::make_shared(header.getColumnsWithTypeAndName()); + auto dag = std::make_unique(header.getColumnsWithTypeAndName()); auto & outputs = dag->getOutputs(); if (use_nulls) @@ -59,7 +59,7 @@ ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, b grouping_node = &dag->materializeNode(*grouping_node); outputs.insert(outputs.begin(), grouping_node); - auto expression = std::make_shared(dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); return std::make_shared(header, expression); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 0ccb0c4492a..90ac94a1ace 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -30,13 +30,13 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDA input_stream_, ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(actions_dag_) + , actions_dag(actions_dag_->clone()) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); + auto expression = std::make_shared(actions_dag->clone(), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -49,7 +49,7 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu pipeline.getHeader().getColumnsWithTypeAndName(), output_stream->header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto convert_actions = std::make_shared(convert_actions_dag, settings.getActionsSettings()); + auto convert_actions = std::make_shared(std::move(convert_actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } @@ -79,7 +79,7 @@ void ExpressionStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(actions_dag); + FindAliasForInputName alias_finder(*actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 3eef14ac129..ebbac8217cb 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -5,7 +5,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class ExpressionTransform; class JoiningTransform; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 72934665b5c..ef9f1d17822 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -46,7 +46,6 @@ FilterStep::FilterStep( filter_column_name_, remove_filter_column_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_)) - , actions_dag(actions_dag_) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { @@ -56,7 +55,7 @@ FilterStep::FilterStep( void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { @@ -70,7 +69,7 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ pipeline.getHeader().getColumnsWithTypeAndName(), output_stream->header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto convert_actions = std::make_shared(convert_actions_dag, settings.getActionsSettings()); + auto convert_actions = std::make_shared(std::move(convert_actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -88,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); expression->describeActions(settings.out, prefix); } @@ -97,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } @@ -111,7 +110,7 @@ void FilterStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(actions_dag); + FindAliasForInputName alias_finder(*actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index 939d0900c86..0f894a570b7 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -5,7 +5,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Implements WHERE, HAVING operations. See FilterTransform. class FilterStep : public ITransformingStep diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 0a3a4094a66..87e16b5a244 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -10,7 +10,7 @@ namespace DB::QueryPlanOptimizations { /// build actions DAG from stack of steps -static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) +static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) return nullptr; @@ -27,10 +27,10 @@ static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_st } static std::set -getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) +getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) { auto actions = buildActionsForPlanPath(dag_stack); - FindOriginalNodeForOutputName original_node_finder(actions); + FindOriginalNodeForOutputName original_node_finder(*actions); std::set original_distinct_columns; for (const auto & column : distinct_columns) { @@ -65,7 +65,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) /// (3) gather actions DAG to find original names for columns in distinct step later std::vector steps_to_update; QueryPlan::Node * node = parent_node; - std::vector dag_stack; + std::vector dag_stack; while (!node->children.empty()) { auto * step = dynamic_cast(node->step.get()); @@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) steps_to_update.push_back(step); if (const auto * const expr = typeid_cast(step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 263598bdca7..ff1cefff09a 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -611,7 +611,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{filter->getExpression(), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{filter->getExpression()->clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 1badd315200..13b691da888 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = filter_step->getExpression(); + auto filter_expression = filter_step->getExpression()->clone(); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index dbcaf5f00a7..e57d3319076 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -18,15 +18,15 @@ void optimizePrimaryKeyCondition(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions, storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter, storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) - source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(filter_step->getExpression()->clone(), filter_step->getFilterColumnName()); /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 537555afa2a..8e782e68db8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 70327bc95b4..7c45ef48252 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -486,7 +486,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), - (dag.filter_node ? dag.dag : nullptr), + (dag.filter_node ? dag.dag.get() : nullptr), parts, max_added_blocks.get(), context); @@ -675,7 +675,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu query_info, context, max_added_blocks, - candidate.dag); + candidate.dag.get()); if (!analyzed) continue; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 0af3869ccf1..c7e96d66817 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -172,7 +172,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod query_info, context, max_added_blocks, - query.filter_node ? query.dag : nullptr); + query.filter_node ? query.dag.get() : nullptr); if (!analyzed) continue; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index af1578d6af8..d8b40b22904 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -213,7 +213,7 @@ bool analyzeProjectionCandidate( const SelectQueryInfo & query_info, const ContextPtr & context, const std::shared_ptr & max_added_blocks, - const ActionsDAGPtr & dag) + const ActionsDAG * dag) { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; @@ -238,7 +238,7 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = dag; + projection_query_info.filter_actions_dag = dag->clone(); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index e1e106b988e..59ad3a43b97 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -60,6 +60,6 @@ bool analyzeProjectionCandidate( const SelectQueryInfo & query_info, const ContextPtr & context, const std::shared_ptr & max_added_blocks, - const ActionsDAGPtr & dag); + const ActionsDAG * dag); } diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 51df25b35f4..d3c75c988e7 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -65,7 +65,7 @@ namespace } /// build actions DAG from stack of steps - ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) + ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) return nullptr; @@ -83,7 +83,7 @@ namespace } bool compareAggregationKeysWithDistinctColumns( - const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector> actions_chain) + const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector> actions_chain) { logDebug("aggregation_keys", aggregation_keys); logDebug("aggregation_keys size", aggregation_keys.size()); @@ -93,7 +93,8 @@ namespace std::set source_columns; for (auto & actions : actions_chain) { - FindOriginalNodeForOutputName original_node_finder(buildActionsForPlanPath(actions)); + auto tmp_actions = buildActionsForPlanPath(actions); + FindOriginalNodeForOutputName original_node_finder(*tmp_actions); for (const auto & column : current_columns) { logDebug("distinct column name", column); @@ -152,8 +153,8 @@ namespace const DistinctStep * distinct_step = typeid_cast(distinct_node->step.get()); chassert(distinct_step); - std::vector dag_stack; - std::vector> actions_chain; + std::vector dag_stack; + std::vector> actions_chain; const DistinctStep * inner_distinct_step = nullptr; const IQueryPlanStep * aggregation_before_distinct = nullptr; const QueryPlan::Node * node = distinct_node; @@ -182,9 +183,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); if (inner_distinct_step = typeid_cast(node->step.get()); inner_distinct_step) @@ -222,7 +223,7 @@ namespace chassert(distinct_step); const auto distinct_columns = getDistinctColumns(distinct_step); - std::vector dag_stack; + std::vector dag_stack; const DistinctStep * inner_distinct_step = nullptr; const QueryPlan::Node * node = distinct_node; while (!node->children.empty()) @@ -235,9 +236,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); inner_distinct_step = typeid_cast(node->step.get()); @@ -267,7 +268,7 @@ namespace logActionsDAG("distinct pass: merged DAG", path_actions); /// compare columns of two DISTINCTs - FindOriginalNodeForOutputName original_node_finder(path_actions); + FindOriginalNodeForOutputName original_node_finder(*path_actions); for (const auto & column : distinct_columns) { const auto * alias_node = original_node_finder.find(String(column)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b2d8aa0e218..e5370c1c130 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -174,7 +174,7 @@ static void updateSortDescriptionForOutputStream( { if (prewhere_info->prewhere_actions) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + FindOriginalNodeForOutputName original_column_finder(*prewhere_info->prewhere_actions); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -185,7 +185,7 @@ static void updateSortDescriptionForOutputStream( if (prewhere_info->row_level_filter) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->row_level_filter); + FindOriginalNodeForOutputName original_column_finder(*prewhere_info->row_level_filter); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -830,10 +830,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ pipes[0].getHeader().getColumnsWithTypeAndName(), pipes[1].getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); + auto converting_expr = std::make_shared(std::move(conversion_action)); pipes[0].addSimpleTransform( - [conversion_action](const Block & header) + [converting_expr](const Block & header) { - auto converting_expr = std::make_shared(conversion_action); return std::make_shared(header, converting_expr); }); return Pipe::unitePipes(std::move(pipes)); @@ -849,7 +849,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ static ActionsDAGPtr createProjection(const Block & header) { - return std::make_shared(header.getNamesAndTypesList()); + return std::make_unique(header.getNamesAndTypesList()); } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( @@ -1046,7 +1046,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( for (size_t j = 0; j < prefix_size; ++j) sort_description.emplace_back(sorting_columns[j], input_order_info->direction); - auto sorting_key_expr = std::make_shared(sorting_key_prefix_expr); + auto sorting_key_expr = std::make_shared(std::move(sorting_key_prefix_expr)); auto merge_streams = [&](Pipe & pipe) { @@ -1341,10 +1341,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( pipes[0].getHeader().getColumnsWithTypeAndName(), pipes[1].getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); + auto converting_expr = std::make_shared(std::move(conversion_action)); pipes[0].addSimpleTransform( - [conversion_action](const Block & header) + [converting_expr](const Block & header) { - auto converting_expr = std::make_shared(conversion_action); return std::make_shared(header, converting_expr); }); return Pipe::unitePipes(std::move(pipes)); @@ -1378,7 +1378,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( static void buildIndexes( std::optional & indexes, - ActionsDAGPtr filter_actions_dag, + const ActionsDAG * filter_actions_dag, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, const ContextPtr & context, @@ -1518,11 +1518,11 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. - query_info.filter_actions_dag = filter_actions_dag; + query_info.filter_actions_dag = std::move(filter_actions_dag); buildIndexes( indexes, - filter_actions_dag, + query_info.filter_actions_dag.get(), data, prepared_parts, context, @@ -1564,7 +1564,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & primary_key_column_names = primary_key.column_names; if (!indexes) - buildIndexes(indexes, query_info_.filter_actions_dag, data, parts, context_, query_info_, metadata_snapshot); + buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot); if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); @@ -1993,7 +1993,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(result.sampling.filter_expression); + auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2031,7 +2031,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(result_projection); + auto projection_actions = std::make_shared(result_projection->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2048,7 +2048,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons ActionsDAG::MatchColumnsMode::Name, true); - auto converting_dag_expr = std::make_shared(convert_actions_dag); + auto converting_dag_expr = std::make_shared(std::move(convert_actions_dag)); pipe.addSimpleTransform([&](const Block & header) { @@ -2126,7 +2126,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -2135,7 +2135,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -2161,7 +2161,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2171,7 +2171,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index caa8aa2e1bd..e32507e1f22 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -23,7 +23,7 @@ struct MergeTreeDataSelectSamplingData bool read_nothing = false; Float64 used_sample_factor = 1.0; std::shared_ptr filter_function; - ActionsDAGPtr filter_expression; + std::shared_ptr filter_expression; }; struct UsefulSkipIndexes diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 11371578c79..b9b239c721b 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -441,7 +441,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() chassert(numbers_storage.step != UInt64{0}); /// Build rpn of query filters - KeyCondition condition(filter_actions_dag, context, column_names, key_expression); + KeyCondition condition(filter_actions_dag.get(), context, column_names, key_expression); if (condition.extractPlainRanges(ranges)) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index ad0940b90b9..b91debc8239 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 126d4824fff..8ac0cc24ed1 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -33,6 +33,7 @@ public: } const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; } + ActionsDAGPtr detachFilterActionsDAG() { return std::move(filter_actions_dag); } const SelectQueryInfo & getQueryInfo() const { return query_info; } const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } @@ -53,7 +54,7 @@ public: void applyFilters() { applyFilters(std::move(filter_nodes)); - filter_dags = {}; + filter_dags.clear(); } virtual void applyFilters(ActionDAGNodes added_filter_nodes); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index ac5e144bf4a..45de6c31d24 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(actions_dag_) + , actions_dag(actions_dag_->clone()) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(actions_dag, settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(actions_dag->clone(), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index a81bc7bb1a9..52ef5437701 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -6,7 +6,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; enum class TotalsMode : uint8_t; diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index 74a0e5930c7..47883e5edf6 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -7,7 +7,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class WindowTransform; diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index ee155d6f78c..fcf576637ff 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -16,13 +16,13 @@ protected: /// Represents pushed down filters in source std::shared_ptr key_condition; - void setKeyConditionImpl(const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & keys) + void setKeyConditionImpl(const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & keys) { key_condition = std::make_shared( filter_actions_dag, context, keys.getNames(), - std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); + std::make_shared(std::make_unique(keys.getColumnsWithTypeAndName()))); } public: diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index bb38c3e1dc5..95267bc24e0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(interpolate_description->actions); + interpolate_actions = std::make_shared(interpolate_description->actions->clone()); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 28d8128e052..2cd51259549 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -516,7 +516,7 @@ void StorageHive::initMinMaxIndexExpression() partition_names = partition_name_types.getNames(); partition_types = partition_name_types.getTypes(); partition_minmax_idx_expr = std::make_shared( - std::make_shared(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); + std::make_unique(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); } NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); @@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression() hivefile_name_types.push_back(column); } hivefile_minmax_idx_expr = std::make_shared( - std::make_shared(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); + std::make_unique(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); } ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) @@ -647,7 +647,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(filter_actions_dag.get(), getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -715,7 +715,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(filter_actions_dag.get(), getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index 2a697fa5654..e03ecc05064 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -160,7 +160,7 @@ KeyDescription KeyDescription::buildEmptyKey() { KeyDescription result; result.expression_list_ast = std::make_shared(); - result.expression = std::make_shared(std::make_shared(), ExpressionActionsSettings{}); + result.expression = std::make_shared(std::make_unique(), ExpressionActionsSettings{}); return result; } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f8cf19120c7..48ec5529af0 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -691,7 +691,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { - auto res = std::make_shared(); + auto res = std::make_unique(); std::unordered_map to_inverted; @@ -777,7 +777,7 @@ void KeyCondition::getAllSpaceFillingCurves() } KeyCondition::KeyCondition( - ActionsDAGPtr filter_dag, + const ActionsDAG * filter_dag, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 2bc3b108e02..14ef74ea113 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -41,7 +41,7 @@ class KeyCondition public: /// Construct key condition from ActionsDAG nodes KeyCondition( - ActionsDAGPtr filter_dag, + const ActionsDAG * filter_dag, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e0ea4cdbcd..7b642c34f37 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -487,7 +487,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP ASTPtr expression_ast; ConditionSelectivityEstimator result; - PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context); + PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag.get(), local_context); if (partition_pruner.isUseless()) { @@ -746,7 +746,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti if (!partition_key.column_names.empty()) partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); - return std::make_shared(std::make_shared(partition_key_columns), settings); + return std::make_shared(std::make_unique(partition_key_columns), settings); } Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) @@ -1152,7 +1152,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (!virtual_columns_block.has(input->result_name)) valid = false; - PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */); + PartitionPruner partition_pruner(metadata_snapshot, filter_dag.get(), local_context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; @@ -6819,7 +6819,7 @@ using PartitionIdToMaxBlock = std::unordered_map; Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6f736a4afd..52916d85fef 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -403,7 +403,7 @@ public: Block getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2e287ff3042..61b8b6fdaa8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -442,7 +442,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( } void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( - std::optional & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context) + std::optional & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context) { if (!filter_dag) return; @@ -463,10 +463,10 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( return; part_offset_condition.emplace(KeyCondition{ - dag, + dag.get(), context, sample.getNames(), - std::make_shared(std::make_shared(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), + std::make_shared(std::make_unique(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), {}}); } @@ -474,7 +474,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context) { if (!filter_dag) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 788355c1e59..39bff5eacd6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -161,7 +161,7 @@ public: /// If possible, construct optional key condition from predicates containing _part_offset column. static void buildKeyConditionFromPartOffset( - std::optional & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context); + std::optional & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context); /// If possible, filter using expression on virtual columns. /// Example: SELECT count() FROM table WHERE _part = 'part_name' @@ -170,7 +170,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context); /// Filter parts using minmax index and partition key. diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e492ca0aec2..457c85eaa46 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -332,7 +332,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec return std::make_shared(index, query, distance_function, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index d511ab84859..282920c608e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -99,7 +99,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; bool isVectorSearch() const override { return true; } private: diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index fc5147bb56c..c6a00751f25 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -201,7 +201,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & } MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) + const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) : WithContext(context_), header(header_), hash_functions(hash_functions_) { if (!filter_actions_dag) @@ -897,7 +897,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con return std::make_shared(bits_per_row, hash_functions, index.column_names); } -MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(filter_actions_dag, context, index.sample_block, hash_functions); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index d66c4b8b6ca..bd1b137176a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -69,7 +69,7 @@ public: std::vector> predicate; }; - MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); + MergeTreeIndexConditionBloomFilter(const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); bool alwaysUnknownOrTrue() const override; @@ -142,7 +142,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const override; private: size_t bits_per_row; diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 8cf58687125..5b6813d12e3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t } MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -733,7 +733,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator } MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition( - const ActionsDAGPtr & filter_dag, ContextPtr context) const + const ActionsDAG * filter_dag, ContextPtr context) const { return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h index 6fd969030df..fe042884550 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h @@ -62,7 +62,7 @@ class MergeTreeConditionBloomFilterText final : public IMergeTreeIndexCondition { public: MergeTreeConditionBloomFilterText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -163,7 +163,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_dag, ContextPtr context) const override; + const ActionsDAG * filter_dag, ContextPtr context) const override; BloomFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 47ce24b91eb..cd6af68ebcc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -186,7 +186,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, } MergeTreeConditionFullText::MergeTreeConditionFullText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & index_sample_block, const GinFilterParameters & params_, @@ -768,7 +768,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregatorForPart } MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(filter_actions_dag, context, index.sample_block, params, token_extractor.get()); }; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index 1a5e848e5ac..8e0b1a22acb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -63,7 +63,7 @@ class MergeTreeConditionFullText final : public IMergeTreeIndexCondition, WithCo { public: MergeTreeConditionFullText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const GinFilterParameters & params_, @@ -170,7 +170,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const override; GinFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 0995e2724ec..cd8065ecadf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -79,7 +79,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(cons } MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( - const ActionsDAGPtr &, ContextPtr) const + const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 130e708d76f..e60335fe724 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -69,7 +69,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 20dfed8cf8f..c60d63a59ba 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -157,7 +157,7 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s namespace { -KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) +KeyCondition buildCondition(const IndexDescription & index, const ActionsDAG * filter_actions_dag, ContextPtr context) { return KeyCondition{filter_actions_dag, context, index.column_names, index.expression}; } @@ -165,7 +165,7 @@ KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr } MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax( - const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) + const IndexDescription & index, const ActionsDAG * filter_actions_dag, ContextPtr context) : index_data_types(index.data_types) , condition(buildCondition(index, filter_actions_dag, context)) { @@ -198,7 +198,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const Me } MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(index, filter_actions_dag, context); } diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index dca26fb7b28..c5031ccbb27 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -50,7 +50,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition public: MergeTreeIndexConditionMinMax( const IndexDescription & index, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -77,7 +77,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; const char* getSerializedFileExtension() const override { return ".idx2"; } MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index b11cbf1e034..7c65381b05b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -245,7 +245,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context) : index_name(index_name_) , max_rows(max_rows_) @@ -272,9 +272,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); - actions = std::make_shared(filter_actions_dag); actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; + actions = std::make_shared(std::move(filter_actions_dag)); } bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const @@ -544,7 +544,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge } MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context); } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 6efc2effafd..abd40b3cf9d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -83,7 +83,7 @@ public: const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -138,7 +138,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; size_t max_rows = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index c9df7210569..59a4b0fbf9c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -367,7 +367,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel return std::make_shared(index, query, distance_function, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index 5107cfee371..41de94402c9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -101,7 +101,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; bool isVectorSearch() const override { return true; } private: diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index a9f1fa9378f..1be73e1c811 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -167,7 +167,7 @@ struct IMergeTreeIndex } virtual MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const = 0; + const ActionsDAG * filter_actions_dag, ContextPtr context) const = 0; virtual bool isVectorSearch() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 78b67de1a7e..8fa5b2cc955 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->row_level_filter, actions_settings), + .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 02f8d6f4f6a..98b35a3ca2c 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -376,7 +376,7 @@ public: { const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - KeyCondition key_condition(filter, context, primary_key_column_names, primary_key.expression); + KeyCondition key_condition(filter.get(), context, primary_key_column_names, primary_key.expression); LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) @@ -437,7 +437,7 @@ void createReadFromPartStep( auto reading = std::make_unique(type, storage, storage_snapshot, std::move(data_part), std::move(columns_to_read), apply_deleted_mask, - filter, std::move(context), log); + std::move(filter), std::move(context), log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 43e3b0c505a..25596b42951 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -50,7 +50,7 @@ void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map; const ActionsDAG::Node & addClonedDAGToDAG( size_t step, const ActionsDAG::Node * original_dag_node, - ActionsDAGPtr new_dag, + const ActionsDAGPtr & new_dag, OriginalToNewNodeMap & node_remap, NodeNameToLastUsedStepMap & node_to_step_map) { @@ -72,7 +72,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { /// If the node is already in the new DAG, return it const auto & node_ref = node_remap.at(node_name); - if (node_ref.dag == new_dag) + if (node_ref.dag == new_dag.get()) return *node_ref.node; /// If the node is known from the previous steps, add it as an input, except for constants @@ -80,7 +80,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { node_ref.dag->addOrReplaceInOutputs(*node_ref.node); const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); - node_remap[node_name] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is it always correct? + node_remap[node_name] = {new_dag.get(), &new_node}; /// TODO: here we update the node reference. Is it always correct? /// Remember the index of the last step which reuses this node. /// We cannot remove this node from the outputs before that step. @@ -93,7 +93,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( if (original_dag_node->type == ActionsDAG::ActionType::INPUT) { const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -102,7 +102,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { const auto & new_node = new_dag->addColumn( ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -110,7 +110,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { const auto & alias_child = addClonedDAGToDAG(step, original_dag_node->children[0], new_dag, node_remap, node_to_step_map); const auto & new_node = new_dag->addAlias(alias_child, original_dag_node->result_name); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -125,7 +125,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( } const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -133,13 +133,13 @@ const ActionsDAG::Node & addClonedDAGToDAG( } const ActionsDAG::Node & addFunction( - ActionsDAGPtr new_dag, + const ActionsDAGPtr & new_dag, const FunctionOverloadResolverPtr & function, ActionsDAG::NodeRawConstPtrs children, OriginalToNewNodeMap & node_remap) { const auto & new_node = new_dag->addFunction(function, children, ""); - node_remap[new_node.result_name] = {new_dag, &new_node}; + node_remap[new_node.result_name] = {new_dag.get(), &new_node}; return new_node; } @@ -147,7 +147,7 @@ const ActionsDAG::Node & addFunction( /// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, /// but it might be required for further steps with its original uncasted type. const ActionsDAG::Node & addCast( - ActionsDAGPtr dag, + const ActionsDAGPtr & dag, const ActionsDAG::Node & node_to_cast, const String & type_name, OriginalToNewNodeMap & node_remap) @@ -173,7 +173,7 @@ const ActionsDAG::Node & addCast( /// 1. produces a result with the proper Nullable or non-Nullable UInt8 type and /// 2. makes sure that the result contains only 0 or 1 values even if the source column contains non-boolean values. const ActionsDAG::Node & addAndTrue( - ActionsDAGPtr dag, + const ActionsDAGPtr & dag, const ActionsDAG::Node & filter_node_to_normalize, OriginalToNewNodeMap & node_remap) { @@ -258,7 +258,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction for (size_t step_index = 0; step_index < condition_groups.size(); ++step_index) { const auto & condition_group = condition_groups[step_index]; - ActionsDAGPtr step_dag = std::make_shared(); + ActionsDAGPtr step_dag = std::make_unique(); String result_name; std::vector new_condition_nodes; @@ -299,7 +299,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } } - steps.push_back({step_dag, result_name}); + steps.push_back({std::move(step_dag), result_name}); } /// 6. Find all outputs of the original DAG @@ -345,11 +345,11 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { for (size_t step_index = 0; step_index < steps.size(); ++step_index) { - const auto & step = steps[step_index]; + auto & step = steps[step_index]; PrewhereExprStep new_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(step.actions, actions_settings), + .actions = std::make_shared(std::move(step.actions), actions_settings), .filter_column_name = step.column_name, /// Don't remove if it's in the list of original outputs .remove_filter_column = diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 9de7b238f57..6df7b5aa054 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -4,7 +4,7 @@ namespace DB { -PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict) +PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, const ActionsDAG * filter_actions_dag, ContextPtr context, bool strict) : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */) , useless((strict && partition_condition.isRelaxed()) || partition_condition.alwaysUnknownOrTrue()) diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index ca24559ca01..d89dfb7b245 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -13,7 +13,7 @@ namespace DB class PartitionPruner { public: - PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict = false); + PartitionPruner(const StorageMetadataPtr & metadata, const ActionsDAG * filter_actions_dag, ContextPtr context, bool strict = false); bool canBePruned(const IMergeTreeDataPart & part) const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2fc6993369d..f640fb9ba0a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -79,7 +79,7 @@ StorageObjectStorageSource::~StorageObjectStorageSource() void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); + setKeyConditionImpl(filter_actions_dag.get(), context_, read_from_format_info.format_header); } std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 52b6674c93d..654b8b788fe 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -19,7 +19,7 @@ class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; struct PrewhereInfo; using PrewhereInfoPtr = std::shared_ptr; @@ -192,7 +192,7 @@ struct SelectQueryInfo ASTPtr parallel_replica_custom_key_ast; /// Filter actions dag for current storage - ActionsDAGPtr filter_actions_dag; + std::shared_ptr filter_actions_dag; ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a3f6b6afc5d..9bddf4f0230 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->row_level_filter, actions_settings), + std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->prewhere_actions, actions_settings), + std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5048ef4788e..be421e8e2bc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1120,7 +1120,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) if (!source) return nullptr; - return source->getFilterActionsDAG(); + return source->detachFilterActionsDAG(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index aaf84f6f82c..702c257bfb6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1235,7 +1235,7 @@ StorageFileSource::~StorageFileSource() void StorageFileSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); + setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e2f92f08d7a..c42e3058347 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1235,7 +1235,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(actions_dag, + filter_actions = std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1273,12 +1273,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(actions_dag, filter_column_name); + step->addFilter(actions_dag->clone(), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag, filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag->clone(), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } @@ -1471,7 +1471,7 @@ void ReadFromMerge::convertAndFilterSourceStream( { pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - auto actions_dag = std::make_shared(pipe_columns); + auto actions_dag = std::make_unique(pipe_columns); QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context); query_tree->setAlias(alias.name); @@ -1486,7 +1486,7 @@ void ReadFromMerge::convertAndFilterSourceStream( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index f550ccb2bc4..c336f597f41 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -187,7 +187,7 @@ public: void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override { - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); + setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); } Chunk generate() override; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 894b1404a21..4d73f8e5c87 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -48,14 +48,14 @@ Pipe StorageValues::read( if (!prepared_pipe.empty()) { - auto dag = std::make_shared(prepared_pipe.getHeader().getColumnsWithTypeAndName()); + auto dag = std::make_unique(prepared_pipe.getHeader().getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(column_names.size()); for (const auto & name : column_names) outputs.push_back(dag->getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); dag->getOutputs().swap(outputs); - auto expression = std::make_shared(dag); + auto expression = std::make_shared(std::move(dag)); prepared_pipe.addSimpleTransform([&](const Block & header) { diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 016de94c17c..2c0d5c5ca85 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -177,7 +177,7 @@ void StorageView::read( /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - auto materializing_actions = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto materializing_actions = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); materializing_actions->addMaterializingOutputActions(); auto materializing = std::make_unique(query_plan.getCurrentDataStream(), std::move(materializing_actions)); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f831465277d..56f65b57367 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -183,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType dag->getOutputs() = {col}; dag->removeUnusedActions(); - result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context_copy)); + result.expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); return result; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..6f7d1d4c39f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,10 +77,10 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) } } -void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) +void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(dag); + auto actions = std::make_shared(dag->clone()); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..0cf8470bc60 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,7 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context); +void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context); /// Builds sets used by ActionsDAG inplace. void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8bca1c97aad..da4e751a88a 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -567,7 +567,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) builder.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(filter_expression), filter_function->getColumnName(), true); + header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); }); /// Adding window column @@ -592,7 +592,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) new_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); @@ -700,7 +700,7 @@ inline void StorageWindowView::fire(UInt32 watermark) getTargetTable()->getInMemoryMetadataPtr()->getColumns(), getContext(), getContext()->getSettingsRef().insert_null_as_default); - auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + auto adding_missing_defaults_actions = std::make_shared(std::move(adding_missing_defaults_dag)); pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, adding_missing_defaults_actions); @@ -711,7 +711,7 @@ inline void StorageWindowView::fire(UInt32 watermark) block_io.pipeline.getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared( - convert_actions_dag, + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); pipe.addSimpleTransform([&](const Block & stream_header) { @@ -1475,7 +1475,7 @@ void StorageWindowView::writeIntoWindowView( pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(filter_expression), + header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); }); } @@ -1583,7 +1583,7 @@ void StorageWindowView::writeIntoWindowView( output->getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto convert_actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, convert_actions); }); } From 5d9d29e37086e19dba852097e7820aff83072a8d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 15:00:07 +0000 Subject: [PATCH 009/321] Fixing some crashes --- src/Interpreters/ActionsDAG.cpp | 15 ++++++++----- src/Interpreters/ActionsDAG.h | 5 +++-- src/Interpreters/ExpressionActions.cpp | 8 ++++--- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 14 ++++++------ src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Planner/Planner.cpp | 18 ++++++++------- src/Planner/PlannerJoinTree.cpp | 22 +++++++++---------- src/Planner/PlannerJoins.cpp | 4 ++-- src/Planner/PlannerWindowFunctions.cpp | 4 +++- src/Processors/QueryPlan/ExpressionStep.cpp | 8 +++---- src/Processors/QueryPlan/FilterStep.cpp | 6 ++--- .../Optimizations/distinctReadInOrder.cpp | 4 ++-- .../Optimizations/filterPushDown.cpp | 4 ++-- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizePrimaryKeyCondition.cpp | 6 ++--- .../Optimizations/optimizeReadInOrder.cpp | 8 +++---- .../optimizeUseAggregateProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 6 ++--- .../Optimizations/removeRedundantDistinct.cpp | 4 ++-- .../QueryPlan/ReadFromMergeTree.cpp | 16 +++++++------- .../QueryPlan/SourceStepWithFilter.cpp | 8 +++---- src/Processors/QueryPlan/TotalsHavingStep.cpp | 8 +++---- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 ++-- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 ++-- src/Storages/SelectQueryInfo.h | 4 ++-- src/Storages/StorageBuffer.cpp | 8 +++---- src/Storages/StorageMerge.cpp | 6 ++--- src/Storages/VirtualColumnUtils.cpp | 4 ++-- 31 files changed, 112 insertions(+), 102 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 23e1e5ee152..c2626285235 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1246,17 +1246,20 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) return true; } -ActionsDAGPtr ActionsDAG::clone() const +ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) { std::unordered_map old_to_new_nodes; - return clone(old_to_new_nodes); + return ActionsDAG::clone(from, old_to_new_nodes); } -ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const +ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes) { + if (!from) + return nullptr; + auto actions = std::make_unique(); - for (const auto & node : nodes) + for (const auto & node : from->nodes) { auto & copy_node = actions->nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; @@ -1266,10 +1269,10 @@ ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_t for (auto & child : node.children) child = old_to_new_nodes[child]; - for (const auto & output_node : outputs) + for (const auto & output_node : from->outputs) actions->outputs.push_back(old_to_new_nodes[output_node]); - for (const auto & input_node : inputs) + for (const auto & input_node : from->inputs) actions->inputs.push_back(old_to_new_nodes[input_node]); return actions; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 4a840885b6a..7ca3d1c1b0d 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,8 +261,9 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - ActionsDAGPtr clone() const; - ActionsDAGPtr clone(std::unordered_map & old_to_new_nodes) const; + static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } + static ActionsDAGPtr clone(const ActionsDAG * from); + static ActionsDAGPtr clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes); static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 7cbf5afd763..2eca31fc75e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -53,7 +53,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio : project_inputs(project_inputs_) , settings(settings_) { - actions_dag = actions_dag_->clone(); + actions_dag = ActionsDAG::clone(actions_dag_); /// It's important to determine lazy executed nodes before compiling expressions. std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation); @@ -76,15 +76,17 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = actions_dag->clone(copy_map); + copy->actions_dag = ActionsDAG::clone(actions_dag.get(), copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; + for (const auto * input : copy->actions_dag->getInputs()) + copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name)); + copy->num_columns = num_columns; copy->required_columns = required_columns; - copy->input_positions = input_positions; copy->result_positions = result_positions; copy->sample_block = sample_block; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index be00e37c751..6b49365b492 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,7 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - auto dag = prewhere_dag_and_flags->dag.clone(); + auto dag = ActionsDAG::clone(&prewhere_dag_and_flags->dag); ExpressionActions( std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); @@ -1980,7 +1980,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - before_where->dag.clone(), + ActionsDAG::clone(&before_where->dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 64a17a7ba87..71eb7dc64f8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1300,7 +1300,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), result_columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); + ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(* ActionsDAG::clone(actions)), std::move(*conv_dag)); interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } @@ -2042,7 +2042,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(prewhere_info.row_level_filter->clone()), + std::make_shared(ActionsDAG::clone(prewhere_info.row_level_filter)), prewhere_info.row_level_column_name, true); }); } @@ -2050,7 +2050,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(prewhere_info.prewhere_actions->clone()), + header, std::make_shared(ActionsDAG::clone(prewhere_info.prewhere_actions)), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2578,7 +2578,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2752,7 +2752,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2770,7 +2770,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( ActionsDAGPtr dag; if (expression) { - dag = expression->dag.clone(); + dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2819,7 +2819,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 1bb770bf561..704c5ce7d8b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1278,7 +1278,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = step->actions()->dag.clone(); + auto dag = ActionsDAG::clone(&step->actions()->dag); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. @@ -1286,7 +1286,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } else { - auto dag = step->actions()->dag.clone(); + auto dag = ActionsDAG::clone(&step->actions()->dag); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 681ae7e6ac4..dddb7531519 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -333,12 +333,12 @@ void addExpressionStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = expression_actions->dag.clone(); + auto actions = ActionsDAG::clone(&expression_actions->dag); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions.get()); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); + result_actions_to_execute.push_back(expression_step->getExpression().get()); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -348,15 +348,15 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = filter_analysis_result.filter_actions->dag.clone(); + auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions.get()); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), actions, filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); + result_actions_to_execute.push_back(where_step->getExpression().get()); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -556,11 +556,9 @@ void addTotalsHavingStep(QueryPlan & query_plan, ActionsDAGPtr actions; if (having_analysis_result.filter_actions) { - actions = having_analysis_result.filter_actions->dag.clone(); + actions = ActionsDAG::clone(&having_analysis_result.filter_actions->dag); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - - result_actions_to_execute.push_back(actions.get()); } auto totals_having_step = std::make_unique( @@ -573,6 +571,10 @@ void addTotalsHavingStep(QueryPlan & query_plan, settings.totals_mode, settings.totals_auto_threshold, need_finalize); + + if (having_analysis_result.filter_actions) + result_actions_to_execute.push_back(totals_having_step->getActions().get()); + query_plan.addStep(std::move(totals_having_step)); } @@ -1449,7 +1451,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(filters.filter_actions->clone()); + table_expression_data.setFilterActions(ActionsDAG::clone(filters.filter_actions)); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 918cfad703e..16b5e363bfd 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -646,7 +646,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions()->clone(); + table_expression_query_info.filter_actions_dag = ActionsDAG::clone(table_expression_data.getFilterActions()); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -831,7 +831,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions->clone()); + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(row_policy_filter_info.actions)); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) @@ -1178,17 +1178,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); - left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); - left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); - join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); + left_join_tree_query_plan.actions_dags.emplace_back(left_join_expressions_actions_step->getExpression().get()); left_plan.addStep(std::move(left_join_expressions_actions_step)); join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); + right_join_tree_query_plan.actions_dags.emplace_back(right_join_expressions_actions_step->getExpression().get()); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1434,7 +1433,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto result_plan = QueryPlan(); - if (join_algorithm->isFilled()) + bool is_filled_join = join_algorithm->isFilled(); + if (is_filled_join) { auto filled_join_step = std::make_unique( left_plan.getCurrentDataStream(), @@ -1586,8 +1586,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` - for (const auto * action_dag : right_join_tree_query_plan.actions_dags) - left_join_tree_query_plan.actions_dags.emplace_back(action_dag); + if (!is_filled_join) + for (const auto * action_dag : right_join_tree_query_plan.actions_dags) + left_join_tree_query_plan.actions_dags.emplace_back(action_dag); // if (join_clauses_and_actions.left_join_expressions_actions) // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); // if (join_clauses_and_actions.right_join_expressions_actions) @@ -1646,10 +1647,9 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); - join_tree_query_plan.actions_dags.push_back(array_join_action_dag.get()); - auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); + join_tree_query_plan.actions_dags.push_back(array_join_actions->getExpression().get()); plan.addStep(std::move(array_join_actions)); auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 45842c0d705..23b6a805ab9 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -588,10 +588,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = left_join_actions->clone(); + result.left_join_expressions_actions = ActionsDAG::clone(left_join_actions); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = right_join_actions->clone(); + result.right_join_expressions_actions = ActionsDAG::clone(right_join_actions); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 9deceeef9a3..b9e11578dbc 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -157,7 +157,9 @@ std::vector sortWindowDescriptions(const std::vector }; std::vector perm(window_descriptions.size()); - std::iota(perm.begin(), perm.end(), 0U); + for (size_t i = 0; i < perm.size(); ++i) + perm[i] = i; + ::sort(perm.begin(), perm.end(), comparator); return perm; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 90ac94a1ace..50bc2e1533e 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -30,13 +30,13 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDA input_stream_, ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(actions_dag_->clone()) + , actions_dag(ActionsDAG::clone(actions_dag_)) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag->clone(), settings.getActionsSettings()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index ef9f1d17822..7883461f45a 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -49,7 +49,7 @@ FilterStep::FilterStep( , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { - actions_dag = actions_dag->clone(); + actions_dag = ActionsDAG::clone(actions_dag_); actions_dag->removeAliasesForFilter(filter_column_name); } @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 87e16b5a244..6cdc3cb4eb0 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -15,11 +15,11 @@ static ActionsDAGPtr buildActionsForPlanPath(std::vector & d if (dag_stack.empty()) return nullptr; - ActionsDAGPtr path_actions = dag_stack.back()->clone(); + ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = dag_stack.back()->clone(); + ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); path_actions->mergeInplace(std::move(*clone)); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index ff1cefff09a..f26cd79dd97 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -597,7 +597,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - filter->getExpression()->clone(), + ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -611,7 +611,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{filter->getExpression()->clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 35d8b1a35e4..4629bc0af53 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - expression->getExpression()->clone()); + ActionsDAG::clone(expression->getExpression())); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 13b691da888..afe1406b65f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = filter_step->getExpression()->clone(); + auto filter_expression = ActionsDAG::clone(filter_step->getExpression()); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index e57d3319076..e5ded92b105 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -18,15 +18,15 @@ void optimizePrimaryKeyCondition(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) - source_step_with_filter->addFilter(filter_step->getExpression()->clone(), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(ActionsDAG::clone(filter_step->getExpression()), filter_step->getFilterColumnName()); /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 8e782e68db8..a8bd98d7460 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -173,9 +173,9 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) - dag->mergeInplace(std::move(*expression->clone())); + dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); else - dag = expression->clone(); + dag = ActionsDAG::clone(expression); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 7c45ef48252..da057bd25c2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -77,7 +77,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = analysis_result.before_aggregation->dag.clone(); + info.before_aggregation = ActionsDAG::clone(&analysis_result.before_aggregation->dag); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index d8b40b22904..0e2ad96a419 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -67,9 +67,9 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea void QueryDAG::appendExpression(const ActionsDAGPtr & expression) { if (dag) - dag->mergeInplace(std::move(*expression->clone())); + dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); else - dag = expression->clone(); + dag = ActionsDAG::clone(expression); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -238,7 +238,7 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = dag->clone(); + projection_query_info.filter_actions_dag = ActionsDAG::clone(dag); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index d3c75c988e7..81a8a537830 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -70,11 +70,11 @@ namespace if (dag_stack.empty()) return nullptr; - ActionsDAGPtr path_actions = dag_stack.back()->clone(); + ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = dag_stack.back()->clone(); + ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); logActionsDAG("DAG to merge", clone); dag_stack.pop_back(); path_actions->mergeInplace(std::move(*clone)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e5370c1c130..d711b3e8472 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -798,7 +798,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone()); + auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -1211,7 +1211,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone()); + auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); if (prewhere_info) { @@ -1993,7 +1993,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); + auto sampling_actions = std::make_shared(ActionsDAG::clone(result.sampling.filter_expression.get())); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2031,7 +2031,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(result_projection->clone()); + auto projection_actions = std::make_shared(ActionsDAG::clone(result_projection)); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2126,7 +2126,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); expression->describeActions(format_settings.out, prefix); } @@ -2135,7 +2135,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); expression->describeActions(format_settings.out, prefix); } } @@ -2161,7 +2161,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2171,7 +2171,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index b91debc8239..79b225e7f93 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 45de6c31d24..19632b1862f 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(actions_dag_->clone()) + , actions_dag(ActionsDAG::clone(actions_dag_)) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(actions_dag->clone(), settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 95267bc24e0..bbe57fc6441 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(interpolate_description->actions->clone()); + interpolate_actions = std::make_shared(ActionsDAG::clone(interpolate_description->actions)); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 7c65381b05b..8d4ef69b1b9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,7 +265,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = filter_dag->clone(); + auto filter_actions_dag = ActionsDAG::clone(filter_dag); const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); std::unordered_map node_to_result_node; @@ -319,7 +319,7 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node return index_hint.getActions()->getOutputs(); /// Import the DAG and map argument pointers. - ActionsDAGPtr actions_clone = index_hint.getActions()->clone(); + ActionsDAGPtr actions_clone = ActionsDAG::clone(index_hint.getActions()); chassert(storage); result_dag_or_null->mergeNodes(std::move(*actions_clone), storage); return *storage; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 8fa5b2cc955..e924f853524 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), + .actions = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), + .actions = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 654b8b788fe..0b7035504ae 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = row_level_filter->clone(); + prewhere_info->row_level_filter = ActionsDAG::clone(row_level_filter); if (prewhere_actions) - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9bddf4f0230..695b31d0c80 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -312,7 +312,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*actions_dag->clone()), + std::move(*ActionsDAG::clone(actions_dag)), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -321,7 +321,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*actions_dag->clone()), + std::move(*ActionsDAG::clone(actions_dag)), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings), + std::make_shared(ActionsDAG::clone(query_info.prewhere_info->row_level_filter), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings), + std::make_shared(ActionsDAG::clone(query_info.prewhere_info->prewhere_actions), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c42e3058347..d21a6dc20dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - filter_info.actions->clone(), + ActionsDAG::clone(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); @@ -1273,12 +1273,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(actions_dag->clone(), filter_column_name); + step->addFilter(ActionsDAG::clone(actions_dag), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag->clone(), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), ActionsDAG::clone(actions_dag), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 6f7d1d4c39f..1bd5e80a4f9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,7 +80,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(dag->clone()); + auto actions = std::make_shared(ActionsDAG::clone(dag)); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); @@ -318,7 +318,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = index_hint->getActions()->clone(); + auto index_hint_dag = ActionsDAG::clone(index_hint->getActions()); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag->getOutputs()) if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) From 068e1c55545dfc4e004b8b5970b3f237033444cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 15:59:00 +0000 Subject: [PATCH 010/321] Fixing build. --- src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 6ace1b3b5ce..97de69b1134 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -84,12 +84,12 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (child_actions->hasArrayJoin()) return 0; - auto actions = child_actions->clone(); + auto actions = ActionsDAG::clone(child_actions); const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName()); if (child_filter->removesFilterColumn()) removeFromOutputs(*actions, child_filter_node); - actions->mergeInplace(std::move(*parent_actions->clone())); + actions->mergeInplace(std::move(*ActionsDAG::clone(parent_actions))); const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); if (parent_filter->removesFilterColumn()) From 3149e51e9254b268c41ade796d3652d0c2dec8f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 17:36:07 +0000 Subject: [PATCH 011/321] Fix other crashes. --- src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5822e46f9f8..8f39f0da5af 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -563,11 +563,11 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) auto syntax_result = TreeRewriter(getContext()).analyze(filter_function, builder.getHeader().getNamesAndTypesList()); auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, getContext()).getActionsDAG(false); + auto filter_actions = std::make_shared(std::move(filter_expression)); builder.addSimpleTransform([&](const Block & header) { - return std::make_shared( - header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); + return std::make_shared(header, filter_actions, filter_function->getColumnName(), true); }); /// Adding window column From 9be404c9c7b1e14410928a4aef8396664d1e364e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jul 2024 18:02:57 +0000 Subject: [PATCH 012/321] Fix another case. --- src/Planner/PlannerJoinTree.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 16b5e363bfd..604d3366484 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1383,11 +1383,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_clauses_and_actions.mixed_join_expressions_actions) { - left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); + left_join_tree_query_plan.actions_dags.push_back(&mixed_join_expression->getActionsDAG()); } } else if (join_node.isUsingJoinExpression()) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5c5ce4ecc2e..c3fdad3a8f2 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1240,7 +1240,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(std::move(actions_dag), + filter_actions = std::make_shared(ActionsDAG::clone(actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); From 6a06024983a78aaab9b7cbe6e9533255debebdb3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jul 2024 10:25:40 +0000 Subject: [PATCH 013/321] Fix for prewhere optimization. --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index afe1406b65f..f203d831750 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -59,7 +59,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans. QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node; - const auto * filter_step = typeid_cast(filter_node->step.get()); + auto * filter_step = typeid_cast(filter_node->step.get()); if (!filter_step) return; @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = ActionsDAG::clone(filter_step->getExpression()); + auto filter_expression = std::move(filter_step->getExpression()); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) @@ -121,7 +121,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true); + auto split_result = filter_expression->split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. From 41c62ca6636572b4d7654dc0dc329740bb5c7425 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jul 2024 14:44:47 +0000 Subject: [PATCH 014/321] better --- src/Planner/Planner.cpp | 91 +++++++++++---------------------- src/Planner/PlannerJoinTree.cpp | 25 ++++----- src/Planner/PlannerJoinTree.h | 4 +- src/Planner/Utils.cpp | 30 +++++++++++ src/Planner/Utils.h | 3 ++ 5 files changed, 75 insertions(+), 78 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index dddb7531519..16ee6de73c4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -10,7 +10,6 @@ #include #include -#include #include #include @@ -331,14 +330,14 @@ public: void addExpressionStep(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { auto actions = ActionsDAG::clone(&expression_actions->dag); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); - result_actions_to_execute.push_back(expression_step->getExpression().get()); + appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -346,7 +345,7 @@ void addExpressionStep(QueryPlan & query_plan, void addFilterStep(QueryPlan & query_plan, const FilterAnalysisResult & filter_analysis_result, const std::string & step_description, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) @@ -356,7 +355,7 @@ void addFilterStep(QueryPlan & query_plan, actions, filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); - result_actions_to_execute.push_back(where_step->getExpression().get()); + appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -544,7 +543,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -573,7 +572,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, need_finalize); if (having_analysis_result.filter_actions) - result_actions_to_execute.push_back(totals_having_step->getActions().get()); + appendSetsFromActionsDAG(*totals_having_step->getActions(), useful_sets); query_plan.addStep(std::move(totals_having_step)); } @@ -887,7 +886,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, const QueryTreeNodePtr & query_tree, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { const auto & query_node = query_tree->as(); @@ -919,7 +918,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, if (expressions_analysis_result.hasLimitBy()) { const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); - addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute); + addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1057,47 +1056,15 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana } } -void collectSetsFromActionsDAG(const ActionsDAG & dag, std::unordered_set & useful_sets) -{ - for (const auto & node : dag.getNodes()) - { - if (node.column) - { - const IColumn * column = node.column.get(); - if (const auto * column_const = typeid_cast(column)) - column = &column_const->getDataColumn(); - - if (const auto * column_set = typeid_cast(column)) - useful_sets.insert(column_set->getData().get()); - } - - if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") - { - ActionsDAG::NodeRawConstPtrs children; - if (const auto * adaptor = typeid_cast(node.function_base.get())) - { - if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) - { - collectSetsFromActionsDAG(*index_hint->getActions(), useful_sets); - } - } - } - } -} - void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) + const UsefulSets & useful_sets) { auto subqueries = planner_context->getPreparedSets().getSubqueries(); - std::unordered_set useful_sets; - for (const auto * actions_to_execute : result_actions_to_execute) - collectSetsFromActionsDAG(*actions_to_execute, useful_sets); - - auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); subqueries.erase(it, subqueries.end()); @@ -1542,15 +1509,15 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info); - std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); + auto useful_sets = std::move(join_tree_query_plan.useful_sets); for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) { if (table_expression_data.getPrewhereFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions().get()); + appendSetsFromActionsDAG(*table_expression_data.getPrewhereFilterActions(), useful_sets); if (table_expression_data.getRowLevelFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions().get()); + appendSetsFromActionsDAG(*table_expression_data.getRowLevelFilterActions(), useful_sets); } if (query_processing_info.isIntermediateStage()) @@ -1561,7 +1528,7 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info, query_tree, - result_actions_to_execute); + useful_sets); if (expression_analysis_result.hasAggregation()) { @@ -1573,13 +1540,13 @@ void Planner::buildPlanForQueryNode() if (query_processing_info.isFirstStage()) { if (expression_analysis_result.hasWhere()) - addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", useful_sets); if (expression_analysis_result.hasAggregation()) { const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); if (aggregation_analysis_result.before_aggregation_actions) - addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", result_actions_to_execute); + addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets); addAggregationStep(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info); } @@ -1598,7 +1565,7 @@ void Planner::buildPlanForQueryNode() */ const auto & window_analysis_result = expression_analysis_result.getWindow(); if (window_analysis_result.before_window_actions) - addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", result_actions_to_execute); + addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets); } else { @@ -1607,7 +1574,7 @@ void Planner::buildPlanForQueryNode() * now, on shards (first_stage). */ const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) { @@ -1623,7 +1590,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { const auto & sort_analysis_result = expression_analysis_result.getSort(); - addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute); + addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } } @@ -1634,7 +1601,7 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info, query_tree, - result_actions_to_execute); + useful_sets); } if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState()) @@ -1656,14 +1623,14 @@ void Planner::buildPlanForQueryNode() if (query_node.isGroupByWithTotals()) { - addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, result_actions_to_execute); + addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, useful_sets); having_executed = true; } addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node); if (!having_executed && expression_analysis_result.hasHaving()) - addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", useful_sets); } if (query_processing_info.isFromAggregationState()) @@ -1678,16 +1645,16 @@ void Planner::buildPlanForQueryNode() { const auto & window_analysis_result = expression_analysis_result.getWindow(); if (expression_analysis_result.hasAggregation()) - addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", result_actions_to_execute); + addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets); addWindowSteps(query_plan, planner_context, window_analysis_result); } if (expression_analysis_result.hasQualify()) - addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets); const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) { @@ -1703,7 +1670,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { const auto & sort_analysis_result = expression_analysis_result.getSort(); - addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute); + addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } else @@ -1756,7 +1723,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy()) { const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); - addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute); + addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1788,7 +1755,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isToAggregationState()) { const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets); } // For additional_result_filter setting @@ -1796,7 +1763,7 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, useful_sets); query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode(); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 604d3366484..94054588d40 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1181,13 +1181,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); - left_join_tree_query_plan.actions_dags.emplace_back(left_join_expressions_actions_step->getExpression().get()); + appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); left_plan.addStep(std::move(left_join_expressions_actions_step)); join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); - right_join_tree_query_plan.actions_dags.emplace_back(right_join_expressions_actions_step->getExpression().get()); + appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1387,7 +1387,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ mixed_join_expression = std::make_shared( std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); - left_join_tree_query_plan.actions_dags.push_back(&mixed_join_expression->getActionsDAG()); + + appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); } } else if (join_node.isUsingJoinExpression()) @@ -1585,16 +1586,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); - /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` + /// Collect all required actions sets in `left_join_tree_query_plan.useful_sets` if (!is_filled_join) - for (const auto * action_dag : right_join_tree_query_plan.actions_dags) - left_join_tree_query_plan.actions_dags.emplace_back(action_dag); - // if (join_clauses_and_actions.left_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); - // if (join_clauses_and_actions.right_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); - // if (join_clauses_and_actions.mixed_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); + for (const auto & useful_set : right_join_tree_query_plan.useful_sets) + left_join_tree_query_plan.useful_sets.insert(useful_set); auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping); auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping; @@ -1604,7 +1599,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ .query_plan = std::move(result_plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(left_join_tree_query_plan.used_row_policies), - .actions_dags = std::move(left_join_tree_query_plan.actions_dags), + .useful_sets = std::move(left_join_tree_query_plan.useful_sets), .query_node_to_plan_step_mapping = std::move(mapping), }; } @@ -1649,7 +1644,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); - join_tree_query_plan.actions_dags.push_back(array_join_actions->getExpression().get()); + appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets); plan.addStep(std::move(array_join_actions)); auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); @@ -1690,7 +1685,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ .query_plan = std::move(plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(join_tree_query_plan.used_row_policies), - .actions_dags = std::move(join_tree_query_plan.actions_dags), + .useful_sets = std::move(join_tree_query_plan.useful_sets), .query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping), }; } diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index 675079427eb..bc58e802a09 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -11,12 +11,14 @@ namespace DB { +using UsefulSets = std::unordered_set; + struct JoinTreeQueryPlan { QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - std::vector actions_dags{}; + UsefulSets useful_sets; std::unordered_map query_node_to_plan_step_mapping{}; }; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 493ecf5ef53..7ac53e0f8c1 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -11,10 +11,12 @@ #include #include +#include #include #include +#include #include @@ -475,4 +477,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings) return additional_result_filter_ast; } +void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) +{ + for (const auto & node : dag.getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); + + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData()); + } + + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets); + } + } + } + } +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 3172847f053..ae60976a8d6 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, ASTPtr parseAdditionalResultFilter(const Settings & settings); +using UsefulSets = std::unordered_set; +void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets); + } From 7843313f8e09eb018a481b4ba70fcf5fc147105e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 18:20:50 +0200 Subject: [PATCH 015/321] Update PlannerJoinTree.h --- src/Planner/PlannerJoinTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index bc58e802a09..259622b1d50 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -18,7 +18,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - UsefulSets useful_sets; + UsefulSets useful_sets{}; std::unordered_map query_node_to_plan_step_mapping{}; }; From 2257f9a2aee5e8a5c5e178e5f7ccaf269018756a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 17:49:50 +0000 Subject: [PATCH 016/321] Change ActionsDAGPtr to ActionsDAG where possible. --- src/Core/InterpolateDescription.cpp | 6 +- src/Core/InterpolateDescription.h | 7 +- src/Functions/indexHint.h | 6 +- src/Interpreters/ActionsDAG.cpp | 148 +++++++++--------- src/Interpreters/ActionsDAG.h | 46 +++--- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 19 ++- src/Interpreters/ExpressionActions.h | 8 +- src/Interpreters/ExpressionAnalyzer.cpp | 51 +++--- src/Interpreters/ExpressionAnalyzer.h | 6 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- .../IInterpreterUnionOrSelectQuery.cpp | 12 +- src/Interpreters/InterpreterSelectQuery.cpp | 49 +++--- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 14 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Interpreters/TableJoin.cpp | 49 +++--- src/Interpreters/TableJoin.h | 10 +- src/Interpreters/addMissingDefaults.cpp | 24 +-- src/Interpreters/addMissingDefaults.h | 2 +- .../evaluateConstantExpression.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 16 +- src/Interpreters/inplaceBlockConversions.h | 6 +- src/Planner/Planner.cpp | 46 +++--- src/Planner/PlannerActionsVisitor.cpp | 14 +- src/Planner/PlannerContext.h | 2 +- src/Planner/PlannerJoinTree.cpp | 81 +++++----- src/Planner/PlannerJoins.cpp | 56 +++---- src/Planner/PlannerJoins.h | 10 +- src/Planner/Utils.cpp | 10 +- src/Processors/QueryPlan/AggregatingStep.cpp | 16 +- src/Processors/QueryPlan/CubeStep.cpp | 12 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 22 +-- src/Processors/QueryPlan/ExpressionStep.h | 11 +- src/Processors/QueryPlan/FilterStep.cpp | 20 +-- src/Processors/QueryPlan/FilterStep.h | 12 +- .../convertOuterJoinToInnerJoin.cpp | 4 +- .../Optimizations/distinctReadInOrder.cpp | 4 +- .../Optimizations/filterPushDown.cpp | 50 +++--- .../Optimizations/liftUpArrayJoin.cpp | 8 +- .../Optimizations/liftUpFunctions.cpp | 6 +- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../Optimizations/mergeExpressions.cpp | 45 +++--- .../Optimizations/optimizePrewhere.cpp | 19 +-- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +- .../Optimizations/optimizeReadInOrder.cpp | 14 +- .../optimizeUseAggregateProjection.cpp | 30 ++-- .../optimizeUseNormalProjection.cpp | 22 +-- .../Optimizations/projectionsCommon.cpp | 14 +- .../Optimizations/projectionsCommon.h | 4 +- .../Optimizations/removeRedundantDistinct.cpp | 12 +- .../Optimizations/removeRedundantSorting.cpp | 4 +- .../QueryPlan/Optimizations/splitFilter.cpp | 14 +- .../useDataParallelAggregation.cpp | 6 +- src/Processors/QueryPlan/PartsSplitter.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 39 ++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- .../QueryPlan/SourceStepWithFilter.cpp | 8 +- .../QueryPlan/SourceStepWithFilter.h | 6 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 14 +- src/Processors/QueryPlan/TotalsHavingStep.h | 7 +- src/Processors/SourceWithKeyCondition.h | 8 +- .../Transforms/AddingDefaultsTransform.cpp | 2 +- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 24 +-- src/Storages/Hive/StorageHive.h | 10 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 4 +- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/KVStorageUtils.h | 2 +- src/Storages/KeyDescription.cpp | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 38 ++--- src/Storages/MergeTree/MergeTreeIndexSet.h | 6 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeSequentialSource.cpp | 8 +- .../MergeTree/MergeTreeSequentialSource.h | 2 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 4 +- .../MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 4 +- .../StorageObjectStorageSource.cpp | 4 +- .../StorageObjectStorageSource.h | 2 +- .../ReadFinalForExternalReplicaStorage.cpp | 2 +- src/Storages/SelectQueryInfo.h | 12 +- src/Storages/StorageBuffer.cpp | 12 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMerge.cpp | 30 ++-- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageTableFunction.h | 2 +- src/Storages/StorageURL.h | 4 +- src/Storages/StorageValues.cpp | 6 +- src/Storages/StorageView.cpp | 6 +- .../System/StorageSystemStackTrace.cpp | 4 +- src/Storages/TTLDescription.cpp | 8 +- src/Storages/VirtualColumnUtils.cpp | 12 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 110 files changed, 718 insertions(+), 721 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 76bbefdcfd7..86681fdb591 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -13,10 +13,10 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) + InterpolateDescription::InterpolateDescription(ActionsDAG actions_, const Aliases & aliases) : actions(std::move(actions_)) { - for (const auto & name_type : actions->getRequiredColumns()) + for (const auto & name_type : actions.getRequiredColumns()) { if (const auto & p = aliases.find(name_type.name); p != aliases.end()) required_columns_map[p->second->getColumnName()] = name_type; @@ -24,7 +24,7 @@ namespace DB required_columns_map[name_type.name] = name_type; } - for (const ColumnWithTypeAndName & column : actions->getResultColumns()) + for (const ColumnWithTypeAndName & column : actions.getResultColumns()) { std::string name = column.name; if (const auto & p = aliases.find(name); p != aliases.end()) diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 73579aebee4..eeead71d780 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -5,21 +5,20 @@ #include #include #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; using Aliases = std::unordered_map; /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases); + explicit InterpolateDescription(ActionsDAG actions, const Aliases & aliases); - ActionsDAGPtr actions; + ActionsDAG actions; std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_set; /// result block columns diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 8fd7b751760..3ab8a021ae1 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -58,11 +58,11 @@ public: return DataTypeUInt8().createColumnConst(input_rows_count, 1u); } - void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); } - const ActionsDAGPtr & getActions() const { return actions; } + void setActions(ActionsDAG actions_) { actions = std::move(actions_); } + const ActionsDAG & getActions() const { return actions; } private: - ActionsDAGPtr actions; + ActionsDAG actions; }; } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index c2626285235..04be9d23c32 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -961,9 +961,9 @@ NameSet ActionsDAG::foldActionsByProjection( } -ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) +ActionsDAG ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) { - auto dag = std::make_unique(); + ActionsDAG dag; std::unordered_map inputs_mapping; std::unordered_map mapping; struct Frame @@ -1003,9 +1003,9 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name != rename->result_name; const auto & input_name = should_rename ? rename->result_name : new_input->result_name; - mapped_input = &dag->addInput(input_name, new_input->result_type); + mapped_input = &dag.addInput(input_name, new_input->result_type); if (should_rename) - mapped_input = &dag->addAlias(*mapped_input, new_input->result_name); + mapped_input = &dag.addAlias(*mapped_input, new_input->result_name); } node = mapped_input; @@ -1034,7 +1034,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name, frame.node->result_name); - auto & node = dag->nodes.emplace_back(*frame.node); + auto & node = dag.nodes.emplace_back(*frame.node); for (auto & child : node.children) child = mapping[child]; @@ -1049,8 +1049,8 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name != mapped_output->result_name) - mapped_output = &dag->addAlias(*mapped_output, output->result_name); - dag->outputs.push_back(mapped_output); + mapped_output = &dag.addAlias(*mapped_output, output->result_name); + dag.outputs.push_back(mapped_output); } return dag; @@ -1411,7 +1411,7 @@ const ActionsDAG::Node & ActionsDAG::materializeNode(const Node & node) return addAlias(*func, name); } -ActionsDAGPtr ActionsDAG::makeConvertingActions( +ActionsDAG ActionsDAG::makeConvertingActions( const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & result, MatchColumnsMode mode, @@ -1428,7 +1428,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (add_casted_columns && mode != MatchColumnsMode::Name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); - auto actions_dag = std::make_unique(source); + ActionsDAG actions_dag(source); NodeRawConstPtrs projection(num_result_columns); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1436,9 +1436,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( std::unordered_map> inputs; if (mode == MatchColumnsMode::Name) { - size_t input_nodes_size = actions_dag->inputs.size(); + size_t input_nodes_size = actions_dag.inputs.size(); for (size_t pos = 0; pos < input_nodes_size; ++pos) - inputs[actions_dag->inputs[pos]->result_name].push_back(pos); + inputs[actions_dag.inputs[pos]->result_name].push_back(pos); } for (size_t result_col_num = 0; result_col_num < num_result_columns; ++result_col_num) @@ -1451,7 +1451,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( { case MatchColumnsMode::Position: { - src_node = dst_node = actions_dag->inputs[result_col_num]; + src_node = dst_node = actions_dag.inputs[result_col_num]; break; } @@ -1462,7 +1462,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( { const auto * res_const = typeid_cast(res_elem.column.get()); if (ignore_constant_values && res_const) - src_node = dst_node = &actions_dag->addColumn(res_elem); + src_node = dst_node = &actions_dag.addColumn(res_elem); else throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Cannot find column `{}` in source stream, there are only columns: [{}]", @@ -1470,7 +1470,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } else { - src_node = dst_node = actions_dag->inputs[input.front()]; + src_node = dst_node = actions_dag.inputs[input.front()]; input.pop_front(); } break; @@ -1483,7 +1483,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (const auto * src_const = typeid_cast(dst_node->column.get())) { if (ignore_constant_values) - dst_node = &actions_dag->addColumn(res_elem); + dst_node = &actions_dag.addColumn(res_elem); else if (res_const->getField() != src_const->getField()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -1505,7 +1505,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( column.column = DataTypeString().createColumnConst(0, column.name); column.type = std::make_shared(); - const auto * right_arg = &actions_dag->addColumn(std::move(column)); + const auto * right_arg = &actions_dag.addColumn(std::move(column)); const auto * left_arg = dst_node; CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; @@ -1513,13 +1513,13 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( = createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; - dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {}); + dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {}); } if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column))) { NodeRawConstPtrs children = {dst_node}; - dst_node = &actions_dag->addFunction(func_builder_materialize, std::move(children), {}); + dst_node = &actions_dag.addFunction(func_builder_materialize, std::move(children), {}); } if (dst_node->result_name != res_elem.name) @@ -1538,7 +1538,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } else { - dst_node = &actions_dag->addAlias(*dst_node, res_elem.name); + dst_node = &actions_dag.addAlias(*dst_node, res_elem.name); projection[result_col_num] = dst_node; } } @@ -1548,36 +1548,36 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } } - actions_dag->outputs.swap(projection); - actions_dag->removeUnusedActions(false); + actions_dag.outputs.swap(projection); + actions_dag.removeUnusedActions(false); return actions_dag; } -ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) +ActionsDAG ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) { - auto adding_column_action = std::make_unique(); + ActionsDAG adding_column_action; FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); auto column_name = column.name; - const auto * column_node = &adding_column_action->addColumn(std::move(column)); + const auto * column_node = &adding_column_action.addColumn(std::move(column)); NodeRawConstPtrs inputs = {column_node}; - const auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {}); - const auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name)); + const auto & function_node = adding_column_action.addFunction(func_builder_materialize, std::move(inputs), {}); + const auto & alias_node = adding_column_action.addAlias(function_node, std::move(column_name)); - adding_column_action->outputs.push_back(&alias_node); + adding_column_action.outputs.push_back(&alias_node); return adding_column_action; } -ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) +ActionsDAG ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) { first.mergeInplace(std::move(second)); /// Some actions could become unused. Do not drop inputs to preserve the header. first.removeUnusedActions(false); - return std::make_unique(std::move(first)); + return std::move(first); } void ActionsDAG::mergeInplace(ActionsDAG && second) @@ -1970,15 +1970,15 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split second_inputs.push_back(cur.to_second); } - auto first_actions = std::make_unique(); - first_actions->nodes.swap(first_nodes); - first_actions->outputs.swap(first_outputs); - first_actions->inputs.swap(first_inputs); + ActionsDAG first_actions; + first_actions.nodes.swap(first_nodes); + first_actions.outputs.swap(first_outputs); + first_actions.inputs.swap(first_inputs); - auto second_actions = std::make_unique(); - second_actions->nodes.swap(second_nodes); - second_actions->outputs.swap(second_outputs); - second_actions->inputs.swap(second_inputs); + ActionsDAG second_actions; + second_actions.nodes.swap(second_nodes); + second_actions.outputs.swap(second_outputs); + second_actions.inputs.swap(second_inputs); std::unordered_map split_nodes_mapping; if (create_split_nodes_mapping) @@ -2098,7 +2098,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS return res; } -bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) +bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const { const auto * filter_node = tryFindInOutputs(filter_name); if (!filter_node) @@ -2122,7 +2122,7 @@ bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & fi input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name)); } - ActionsDAGPtr filter_with_default_value_inputs; + std::optional filter_with_default_value_inputs; try { @@ -2304,12 +2304,12 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt /// /// Result actions add single column with conjunction result (it is always first in outputs). /// No other columns are added or removed. -ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) +std::optional ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) { if (conjunction.empty()) - return nullptr; + return {}; - auto actions = std::make_unique(); + ActionsDAG actions; FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); @@ -2350,7 +2350,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti if (cur.next_child_to_visit == cur.node->children.size()) { - auto & node = actions->nodes.emplace_back(*cur.node); + auto & node = actions.nodes.emplace_back(*cur.node); nodes_mapping[cur.node] = &node; for (auto & child : node.children) @@ -2373,33 +2373,33 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti for (const auto * predicate : conjunction) args.emplace_back(nodes_mapping[predicate]); - result_predicate = &actions->addFunction(func_builder_and, std::move(args), {}); + result_predicate = &actions.addFunction(func_builder_and, std::move(args), {}); } - actions->outputs.push_back(result_predicate); + actions.outputs.push_back(result_predicate); for (const auto & col : all_inputs) { const Node * input; auto & list = required_inputs[col.name]; if (list.empty()) - input = &actions->addInput(col); + input = &actions.addInput(col); else { input = list.front(); list.pop_front(); - actions->inputs.push_back(input); + actions.inputs.push_back(input); } /// We should not add result_predicate into the outputs for the second time. if (input->result_name != result_predicate->result_name) - actions->outputs.push_back(input); + actions.outputs.push_back(input); } return actions; } -ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( +std::optional ActionsDAG::splitActionsForFilterPushDown( const std::string & filter_name, bool removes_filter, const Names & available_inputs, @@ -2415,7 +2415,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( /// If condition is constant let's do nothing. /// It means there is nothing to push down or optimization was already applied. if (predicate->type == ActionType::COLUMN) - return nullptr; + return {}; std::unordered_set allowed_nodes; @@ -2439,7 +2439,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( auto conjunction = getConjunctionNodes(predicate, allowed_nodes); if (conjunction.allowed.empty()) - return nullptr; + return {}; chassert(predicate->result_type); @@ -2451,13 +2451,13 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( && !conjunction.rejected.front()->result_type->equals(*predicate->result_type)) { /// No further optimization can be done - return nullptr; + return {}; } } auto actions = createActionsForConjunction(conjunction.allowed, all_inputs); if (!actions) - return nullptr; + return {}; /// Now, when actions are created, update the current DAG. removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter); @@ -2562,11 +2562,11 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName()); auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName()); - auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter, + auto replace_equivalent_columns_in_filter = [](const ActionsDAG & filter, const Block & stream_header, const std::unordered_map & columns_to_replace) { - auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace); + auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter.getOutputs()[0]}, columns_to_replace); chassert(updated_filter->getOutputs().size() == 1); /** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from @@ -2587,7 +2587,7 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu for (const auto & input : updated_filter->getInputs()) updated_filter_inputs[input->result_name].push_back(input); - for (const auto & input : filter->getInputs()) + for (const auto & input : filter.getInputs()) { if (updated_filter_inputs.contains(input->result_name)) continue; @@ -2625,12 +2625,12 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu }; if (left_stream_filter_to_push_down) - left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down, + left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*left_stream_filter_to_push_down, left_stream_header, equivalent_right_stream_column_to_left_stream_column); if (right_stream_filter_to_push_down) - right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down, + right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*right_stream_filter_to_push_down, right_stream_header, equivalent_left_stream_column_to_right_stream_column); @@ -2859,13 +2859,13 @@ bool ActionsDAG::isSortingPreserved( return true; } -ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( +std::optional ActionsDAG::buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column, bool single_output_condition_node) { if (filter_nodes.empty()) - return nullptr; + return {}; struct Frame { @@ -2873,7 +2873,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( bool visited_children = false; }; - auto result_dag = std::make_unique(); + ActionsDAG result_dag; std::unordered_map result_inputs; std::unordered_map node_to_result_node; @@ -2904,7 +2904,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { auto & result_input = result_inputs[input_node_it->second.name]; if (!result_input) - result_input = &result_dag->addInput(input_node_it->second); + result_input = &result_dag.addInput(input_node_it->second); node_to_result_node.emplace(node, result_input); nodes_to_process.pop_back(); @@ -2931,25 +2931,25 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { auto & result_input = result_inputs[node->result_name]; if (!result_input) - result_input = &result_dag->addInput({node->column, node->result_type, node->result_name}); + result_input = &result_dag.addInput({node->column, node->result_type, node->result_name}); result_node = result_input; break; } case ActionsDAG::ActionType::COLUMN: { - result_node = &result_dag->addColumn({node->column, node->result_type, node->result_name}); + result_node = &result_dag.addColumn({node->column, node->result_type, node->result_name}); break; } case ActionsDAG::ActionType::ALIAS: { const auto * child = node->children.front(); - result_node = &result_dag->addAlias(*(node_to_result_node.find(child)->second), node->result_name); + result_node = &result_dag.addAlias(*(node_to_result_node.find(child)->second), node->result_name); break; } case ActionsDAG::ActionType::ARRAY_JOIN: { const auto * child = node->children.front(); - result_node = &result_dag->addArrayJoin(*(node_to_result_node.find(child)->second), {}); + result_node = &result_dag.addArrayJoin(*(node_to_result_node.find(child)->second), {}); break; } case ActionsDAG::ActionType::FUNCTION: @@ -2967,13 +2967,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - ActionsDAGPtr index_hint_filter_dag; - const auto & index_hint_args = index_hint->getActions()->getOutputs(); + ActionsDAG index_hint_filter_dag; + const auto & index_hint_args = index_hint->getActions().getOutputs(); - if (index_hint_args.empty()) - index_hint_filter_dag = std::make_unique(); - else - index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, + if (!index_hint_args.empty()) + index_hint_filter_dag = *buildFilterActionsDAG(index_hint_args, node_name_to_input_node_column, false /*single_output_condition_node*/); @@ -2995,7 +2993,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( auto [arguments, all_const] = getFunctionArguments(function_children); auto function_base = function_overload_resolver ? function_overload_resolver->build(arguments) : node->function_base; - result_node = &result_dag->addFunctionImpl( + result_node = &result_dag.addFunctionImpl( function_base, std::move(function_children), std::move(arguments), @@ -3010,7 +3008,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( nodes_to_process.pop_back(); } - auto & result_dag_outputs = result_dag->getOutputs(); + auto & result_dag_outputs = result_dag.getOutputs(); result_dag_outputs.reserve(filter_nodes_size); for (const auto & node : filter_nodes) @@ -3019,7 +3017,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( if (result_dag_outputs.size() > 1 && single_output_condition_node) { FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - result_dag_outputs = { &result_dag->addFunction(func_builder_and, result_dag_outputs, {}) }; + result_dag_outputs = { &result_dag.addFunction(func_builder_and, result_dag_outputs, {}) }; } return result_dag; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 7ca3d1c1b0d..cf6a91b9fe7 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -247,7 +247,7 @@ public: /// c * d e /// \ / /// c * d - e - static ActionsDAGPtr foldActionsByProjection( + static ActionsDAG foldActionsByProjection( const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); @@ -303,7 +303,7 @@ public: /// @param ignore_constant_values - Do not check that constants are same. Use value from result_header. /// @param add_casted_columns - Create new columns with converted values instead of replacing original. /// @param new_names - Output parameter for new column names when add_casted_columns is used. - static ActionsDAGPtr makeConvertingActions( + static ActionsDAG makeConvertingActions( const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & result, MatchColumnsMode mode, @@ -312,13 +312,13 @@ public: NameToNameMap * new_names = nullptr); /// Create expression which add const column and then materialize it. - static ActionsDAGPtr makeAddingColumnActions(ColumnWithTypeAndName column); + static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column); /// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently. /// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression. /// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`. /// Otherwise, any two actions may be combined. - static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second); + static ActionsDAG merge(ActionsDAG && first, ActionsDAG && second); /// The result is similar to merge(*this, second); /// Invariant : no nodes are removed from the first (this) DAG. @@ -329,12 +329,7 @@ public: /// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs(). void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr); - struct SplitResult - { - ActionsDAGPtr first; - ActionsDAGPtr second; - std::unordered_map split_nodes_mapping; - }; + struct SplitResult; /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. /// Execution of first then second parts on block is equivalent to execution of initial DAG. @@ -362,7 +357,7 @@ public: * @param filter_name - name of filter node in current DAG. * @param input_stream_header - input stream header. */ - bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header); + bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const; /// Create actions which may calculate part of filter using only available_inputs. /// If nothing may be calculated, returns nullptr. @@ -381,19 +376,13 @@ public: /// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`. /// To avoid it, add inputs from `all_inputs` list, /// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order. - ActionsDAGPtr splitActionsForFilterPushDown( + std::optional splitActionsForFilterPushDown( const std::string & filter_name, bool removes_filter, const Names & available_inputs, const ColumnsWithTypeAndName & all_inputs); - struct ActionsForJOINFilterPushDown - { - ActionsDAGPtr left_stream_filter_to_push_down; - bool left_stream_filter_removes_filter; - ActionsDAGPtr right_stream_filter_to_push_down; - bool right_stream_filter_removes_filter; - }; + struct ActionsForJOINFilterPushDown; /** Split actions for JOIN filter push down. * @@ -440,7 +429,7 @@ public: * * If single_output_condition_node = false, result dag has multiple output nodes. */ - static ActionsDAGPtr buildFilterActionsDAG( + static std::optional buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column = {}, bool single_output_condition_node = true); @@ -472,11 +461,26 @@ private: void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); + static std::optional createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter); }; +struct ActionsDAG::SplitResult +{ + ActionsDAG first; + ActionsDAG second; + std::unordered_map split_nodes_mapping; +}; + +struct ActionsDAG::ActionsForJOINFilterPushDown +{ + std::optional left_stream_filter_to_push_down; + bool left_stream_filter_removes_filter; + std::optional right_stream_filter_to_push_down; + bool right_stream_filter_removes_filter; +}; + class FindOriginalNodeForOutputName { using NameToNodeIndex = std::unordered_map; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 036b5ba9be0..c2dcdcd34e7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1022,7 +1022,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & dag.project(args); auto index_hint = std::make_shared(); - index_hint->setActions(std::make_unique(std::move(dag))); + index_hint->setActions(std::move(dag)); // Arguments are removed. We add function instead of constant column to avoid constant folding. data.addFunction(std::make_unique(index_hint), {}, column_name); @@ -1285,7 +1285,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - std::make_unique(std::move(lambda_dag)), + std::move(lambda_dag), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 2eca31fc75e..399f4f2ff4f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -49,14 +49,13 @@ namespace ErrorCodes static std::unordered_set processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation); -ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) - : project_inputs(project_inputs_) +ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) + : actions_dag(std::move(actions_dag_)) + , project_inputs(project_inputs_) , settings(settings_) { - actions_dag = ActionsDAG::clone(actions_dag_); - /// It's important to determine lazy executed nodes before compiling expressions. - std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation); + std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation); #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) @@ -68,7 +67,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns) throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, "Too many temporary columns: {}. Maximum: {}", - actions_dag->dumpNames(), settings.max_temporary_columns); + actions_dag.dumpNames(), settings.max_temporary_columns); } ExpressionActionsPtr ExpressionActions::clone() const @@ -76,12 +75,12 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = ActionsDAG::clone(actions_dag.get(), copy_map); + copy->actions_dag = std::move(*ActionsDAG::clone(&actions_dag, copy_map)); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; - for (const auto * input : copy->actions_dag->getInputs()) + for (const auto * input : copy->actions_dag.getInputs()) copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name)); copy->num_columns = num_columns; @@ -357,8 +356,8 @@ void ExpressionActions::linearizeActions(const std::unordered_setgetOutputs(); - const auto & inputs = actions_dag->getInputs(); + const auto & outputs = actions_dag.getOutputs(); + const auto & inputs = actions_dag.getInputs(); auto reverse_info = getActionsDAGReverseInfo(nodes, outputs); std::vector data; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 63ea989bd5e..6ff39ee07f7 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -70,7 +70,7 @@ public: using NameToInputMap = std::unordered_map>; private: - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; Actions actions; size_t num_columns = 0; @@ -84,13 +84,13 @@ private: ExpressionActionsSettings settings; public: - explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); + explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); ExpressionActions(ExpressionActions &&) = default; ExpressionActions & operator=(ExpressionActions &&) = default; const Actions & getActions() const { return actions; } - const std::list & getNodes() const { return actions_dag->getNodes(); } - const ActionsDAG & getActionsDAG() const { return *actions_dag; } + const std::list & getNodes() const { return actions_dag.getNodes(); } + const ActionsDAG & getActionsDAG() const { return actions_dag; } const ColumnNumbers & getResultPositions() const { return result_positions; } const ExpressionActionsSettings & getSettings() const { return settings; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6b49365b492..068b6f290fa 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -928,7 +928,7 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); - ActionsDAGPtr converting_actions; + std::optional converting_actions; JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions); if (converting_actions) @@ -1039,7 +1039,7 @@ static std::unique_ptr buildJoinedPlan( /// Actions which need to be calculated on joined block. auto joined_block_actions = analyzed_join.createJoinedBlockActions(context); NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); Names original_right_column_names; for (auto & pr : required_columns_with_aliases) @@ -1060,17 +1060,17 @@ static std::unique_ptr buildJoinedPlan( interpreter->buildQueryPlan(*joined_plan); { Block original_right_columns = interpreter->getSampleBlock(); - auto rename_dag = std::make_unique(original_right_columns.getColumnsWithTypeAndName()); + ActionsDAG rename_dag(original_right_columns.getColumnsWithTypeAndName()); for (const auto & name_with_alias : required_columns_with_aliases) { if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first)) { auto pos = original_right_columns.getPositionByName(name_with_alias.first); - const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second); - rename_dag->getOutputs()[pos] = &alias; + const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], name_with_alias.second); + rename_dag.getOutputs()[pos] = &alias; } } - rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); + rename_dag.appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); auto rename_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(rename_dag)); rename_step->setStepDescription("Rename joined columns"); joined_plan->addStep(std::move(rename_step)); @@ -1130,14 +1130,14 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a JoinPtr SelectQueryExpressionAnalyzer::makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, - ActionsDAGPtr & left_convert_actions) + std::optional & left_convert_actions) { /// Two JOINs are not supported with the same subquery, but different USINGs. if (joined_plan) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query"); - ActionsDAGPtr right_convert_actions = nullptr; + std::optional right_convert_actions; const auto & analyzed_join = syntax->analyzed_join; @@ -1145,7 +1145,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( { auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext()); NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); Names original_right_column_names; for (auto & pr : required_columns_with_aliases) @@ -1162,7 +1162,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); if (right_convert_actions) { - auto converting_step = std::make_unique(joined_plan->getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); joined_plan->addStep(std::move(converting_step)); } @@ -1354,8 +1354,8 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - auto actions_dag = std::make_unique(columns_after_join); - getRootActions(child, only_types, *actions_dag); + ActionsDAG actions_dag(columns_after_join); + getRootActions(child, only_types, actions_dag); group_by_elements_actions.emplace_back( std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1471,7 +1471,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList()); step.required_output = std::move(required_output); - step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); + step.actions()->dag = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position); } void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) @@ -1607,8 +1607,8 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr { for (const auto & child : select_query->orderBy()->children) { - auto actions_dag = std::make_unique(columns_after_join); - getRootActions(child, only_types, *actions_dag); + ActionsDAG actions_dag(columns_after_join); + getRootActions(child, only_types, actions_dag); order_by_elements_actions.emplace_back( std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1737,7 +1737,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const step.addRequiredOutput(expr->getColumnName()); } -ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) +ActionsDAG ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) { ActionsDAG actions_dag(aggregated_columns); NamesWithAliases result_columns; @@ -1789,7 +1789,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un actions_dag.removeUnusedActions(name_set); } - return std::make_unique(std::move(actions_dag)); + return actions_dag; } ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions) @@ -1798,10 +1798,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result); } -ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) +ActionsDAG ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) { - auto actions = std::make_unique(constant_inputs); - getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); + ActionsDAG actions(constant_inputs); + getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */); return actions; } @@ -1879,8 +1879,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (prewhere_dag_and_flags) { - auto dag = std::make_unique(std::move(prewhere_dag_and_flags->dag)); - prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); + prewhere_info = std::make_shared(std::move(prewhere_dag_and_flags->dag), query.prewhere()->getColumnName()); prewhere_dag_and_flags.reset(); } @@ -1944,7 +1943,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - auto dag = ActionsDAG::clone(&prewhere_dag_and_flags->dag); + ActionsDAG dag = std::move(*ActionsDAG::clone(&prewhere_dag_and_flags->dag)); ExpressionActions( std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); @@ -1980,7 +1979,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - ActionsDAG::clone(&before_where->dag), + std::move(*ActionsDAG::clone(&before_where->dag)), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem @@ -2054,7 +2053,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( auto & step = chain.lastStep(query_analyzer.aggregated_columns); auto & actions = step.actions()->dag; - actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting))); + actions = ActionsDAG::merge(std::move(actions), std::move(converting)); } } @@ -2235,7 +2234,7 @@ void ExpressionAnalysisResult::checkActions() const /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (hasPrewhere()) { - auto check_actions = [](const ActionsDAGPtr & actions) + auto check_actions = [](const std::optional & actions) { if (actions) for (const auto & node : actions->getNodes()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index e44a5891e77..737d36eb504 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -117,12 +117,12 @@ public: /// If add_aliases, only the calculated values in the desired order and add aliases. /// If also remove_unused_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. - ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true); + ActionsDAG getActionsDAG(bool add_aliases, bool remove_unused_result = true); ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no); /// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants. /// Does not execute subqueries. - ActionsDAGPtr getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {}); + ActionsDAG getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {}); ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {}); /** Sets that require a subquery to be create. @@ -367,7 +367,7 @@ private: JoinPtr makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, - ActionsDAGPtr & left_convert_actions); + std::optional & left_convert_actions); const ASTSelectQuery * getAggregatingQuery() const; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 64b6eb5dce9..fcf0d591918 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -295,7 +295,7 @@ private: { auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext()); NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); for (auto & pr : required_columns_with_aliases) required_columns.push_back(pr.first); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index fed29b410db..288d06d2220 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -99,16 +99,16 @@ static ASTPtr parseAdditionalPostFilter(const Context & context) "additional filter", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks); } -static ActionsDAGPtr makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) +static ActionsDAG makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) { auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList()); String result_column_name = ast->getColumnName(); auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false); - const ActionsDAG::Node * result_node = &dag->findInOutputs(result_column_name); - auto & outputs = dag->getOutputs(); + const ActionsDAG::Node * result_node = &dag.findInOutputs(result_column_name); + auto & outputs = dag.getOutputs(); outputs.clear(); - outputs.reserve(dag->getInputs().size() + 1); - for (const auto * node : dag->getInputs()) + outputs.reserve(dag.getInputs().size() + 1); + for (const auto * node : dag.getInputs()) outputs.push_back(node); outputs.push_back(result_node); @@ -126,7 +126,7 @@ void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) c return; auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header); - std::string filter_name = dag->getOutputs().back()->result_name; + std::string filter_name = dag.getOutputs().back()->result_name; auto filter_step = std::make_unique( plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true); filter_step->setStepDescription("Additional result filter"); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 09a7e440f31..cde6e305005 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -175,7 +175,7 @@ FilterDAGInfoPtr generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); - filter_info->actions = std::make_unique(std::move(analyzer.simpleSelectActions()->dag)); + filter_info->actions = std::move(analyzer.simpleSelectActions()->dag); filter_info->column_name = expr_list->children.at(0)->getColumnName(); filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); @@ -938,7 +938,8 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() } } - query_info_copy.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes)) + query_info_copy.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy); /// Note that we treat an estimation of 0 rows as a real estimation size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; @@ -973,7 +974,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) ActionsDAG::MatchColumnsMode::Name, true); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); query_plan.addStep(std::move(converting)); } @@ -1297,10 +1298,10 @@ static InterpolateDescriptionPtr getInterpolateDescription( auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns); ExpressionAnalyzer analyzer(exprs, syntax_result, context); - ActionsDAGPtr actions = analyzer.getActionsDAG(true); - ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), + ActionsDAG actions = analyzer.getActionsDAG(true); + ActionsDAG conv_dag = ActionsDAG::makeConvertingActions(actions.getResultColumns(), result_columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(* ActionsDAG::clone(actions)), std::move(*conv_dag)); + ActionsDAG merge_dag = ActionsDAG::merge(std::move(actions), std::move(conv_dag)); interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } @@ -1485,7 +1486,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.filter_info->actions, + std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1499,7 +1500,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->row_level_filter, + std::move(*ActionsDAG::clone(&*expressions.prewhere_info->row_level_filter)), expressions.prewhere_info->row_level_column_name, true); @@ -1509,7 +1510,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->prewhere_actions, + std::move(*ActionsDAG::clone(&*expressions.prewhere_info->prewhere_actions)), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -1611,7 +1612,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.filter_info->actions, + std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1623,7 +1624,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - new_filter_info->actions, + std::move(*ActionsDAG::clone(&*new_filter_info->actions)), new_filter_info->column_name, new_filter_info->do_remove_column); @@ -2045,7 +2046,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(ActionsDAG::clone(prewhere_info.row_level_filter)), + std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.row_level_filter))), prewhere_info.row_level_column_name, true); }); } @@ -2053,7 +2054,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(ActionsDAG::clone(prewhere_info.prewhere_actions)), + header, std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.prewhere_actions))), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2106,7 +2107,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = std::move(*analysis.filter_info->actions); analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } @@ -2323,7 +2324,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle if (!filter_actions_dag) return {}; - return storage->totalRowsByPartitionPredicate(filter_actions_dag, context); + return storage->totalRowsByPartitionPredicate(*filter_actions_dag, context); } } @@ -2573,7 +2574,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), alias_actions); + auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&*alias_actions))); table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } @@ -2581,9 +2582,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = ActionsDAG::clone(&expression->dag); + auto dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto where_step = std::make_unique( query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter); @@ -2755,9 +2756,9 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = ActionsDAG::clone(&expression->dag); + auto dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto having_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter); @@ -2770,10 +2771,10 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final) { - ActionsDAGPtr dag; + std::optional dag; if (expression) { - dag = ActionsDAG::clone(&expression->dag); + dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2822,9 +2823,9 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - auto dag = ActionsDAG::clone(&expression->dag); + ActionsDAG dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index d4ed19d45ea..ed6dd8af3b2 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -240,7 +240,7 @@ private: Block source_header; /// Actions to calculate ALIAS if required. - ActionsDAGPtr alias_actions; + std::optional alias_actions; /// The subquery interpreter, if the subquery std::unique_ptr interpreter_subquery; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 704c5ce7d8b..2372d26e83f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1197,7 +1197,7 @@ void MutationsInterpreter::Source::read( const auto & names = first_stage.filter_column_names; size_t num_filters = names.size(); - ActionsDAGPtr filter; + std::optional filter; if (!first_stage.filter_column_names.empty()) { ActionsDAG::NodeRawConstPtrs nodes(num_filters); @@ -1278,19 +1278,19 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = ActionsDAG::clone(&step->actions()->dag); + auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); if (step->actions()->project_input) - dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), std::move(dag), stage.filter_column_names[i], false)); } else { - auto dag = ActionsDAG::clone(&step->actions()->dag); + auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); if (step->actions()->project_input) - dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), std::move(dag))); } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4821d607d0e..c1d7acf0775 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -182,7 +182,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, ASTPtr temp_ast = expr.clone(); auto syntax = TreeRewriter(context).analyze(temp_ast, columns); - auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false)->getRequiredColumns(); + auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false).getRequiredColumns(); return required_columns; } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index baf3a743f40..c8c926db13c 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -462,19 +462,19 @@ static void makeColumnNameUnique(const ColumnsWithTypeAndName & source_columns, } } -static ActionsDAGPtr createWrapWithTupleActions( +static std::optional createWrapWithTupleActions( const ColumnsWithTypeAndName & source_columns, std::unordered_set && column_names_to_wrap, NameToNameMap & new_names) { if (column_names_to_wrap.empty()) - return nullptr; + return {}; - auto actions_dag = std::make_unique(source_columns); + ActionsDAG actions_dag(source_columns); FunctionOverloadResolverPtr func_builder = std::make_unique(std::make_shared()); - for (const auto * input_node : actions_dag->getInputs()) + for (const auto * input_node : actions_dag.getInputs()) { const auto & column_name = input_node->result_name; auto it = column_names_to_wrap.find(column_name); @@ -485,9 +485,9 @@ static ActionsDAGPtr createWrapWithTupleActions( String node_name = "__wrapNullsafe(" + column_name + ")"; makeColumnNameUnique(source_columns, node_name); - const auto & dst_node = actions_dag->addFunction(func_builder, {input_node}, node_name); + const auto & dst_node = actions_dag.addFunction(func_builder, {input_node}, node_name); new_names[column_name] = dst_node.result_name; - actions_dag->addOrReplaceInOutputs(dst_node); + actions_dag.addOrReplaceInOutputs(dst_node); } if (!column_names_to_wrap.empty()) @@ -537,21 +537,23 @@ std::pair TableJoin::getKeysForNullSafeComparion(const Columns return {left_keys_to_wrap, right_keys_to_wrap}; } -static void mergeDags(ActionsDAGPtr & result_dag, ActionsDAGPtr && new_dag) +static void mergeDags(std::optional & result_dag, std::optional && new_dag) { + if (!new_dag) + return; if (result_dag) result_dag->mergeInplace(std::move(*new_dag)); else result_dag = std::move(new_dag); } -std::pair +std::pair, std::optional> TableJoin::createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) { - ActionsDAGPtr left_dag = nullptr; - ActionsDAGPtr right_dag = nullptr; + std::optional left_dag; + std::optional right_dag; /** If the types are not equal, we need to convert them to a common type. * Example: * SELECT * FROM t1 JOIN t2 ON t1.a = t2.b @@ -693,7 +695,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } } -static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, +static std::optional changeKeyTypes(const ColumnsWithTypeAndName & cols_src, const TableJoin::NameToTypeMap & type_mapping, bool add_new_cols, NameToNameMap & key_column_rename) @@ -710,7 +712,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, } } if (!has_some_to_do) - return nullptr; + return {}; return ActionsDAG::makeConvertingActions( /* source= */ cols_src, @@ -721,7 +723,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, /* new_names= */ &key_column_rename); } -static ActionsDAGPtr changeTypesToNullable( +static std::optional changeTypesToNullable( const ColumnsWithTypeAndName & cols_src, const NameSet & exception_cols) { @@ -737,7 +739,7 @@ static ActionsDAGPtr changeTypesToNullable( } if (!has_some_to_do) - return nullptr; + return {}; return ActionsDAG::makeConvertingActions( /* source= */ cols_src, @@ -748,29 +750,29 @@ static ActionsDAGPtr changeTypesToNullable( /* new_names= */ nullptr); } -ActionsDAGPtr TableJoin::applyKeyConvertToTable( +std::optional TableJoin::applyKeyConvertToTable( const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, JoinTableSide table_side, NameToNameMap & key_column_rename) { if (type_mapping.empty()) - return nullptr; + return {}; /// Create DAG to convert key columns - ActionsDAGPtr convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename); + auto convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename); applyRename(table_side, key_column_rename); return convert_dag; } -ActionsDAGPtr TableJoin::applyNullsafeWrapper( +std::optional TableJoin::applyNullsafeWrapper( const ColumnsWithTypeAndName & cols_src, const NameSet & columns_for_nullsafe_comparison, JoinTableSide table_side, NameToNameMap & key_column_rename) { if (columns_for_nullsafe_comparison.empty()) - return nullptr; + return {}; std::unordered_set column_names_to_wrap; for (const auto & name : columns_for_nullsafe_comparison) @@ -784,7 +786,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper( } /// Create DAG to wrap keys with tuple for null-safe comparison - ActionsDAGPtr null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename); + auto null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename); for (auto & clause : clauses) { for (size_t i : clause.nullsafe_compare_key_indexes) @@ -799,7 +801,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper( return null_safe_wrap_dag; } -ActionsDAGPtr TableJoin::applyJoinUseNullsConversion( +std::optional TableJoin::applyJoinUseNullsConversion( const ColumnsWithTypeAndName & cols_src, const NameToNameMap & key_column_rename) { @@ -809,8 +811,7 @@ ActionsDAGPtr TableJoin::applyJoinUseNullsConversion( exclude_columns.insert(it.second); /// Create DAG to make columns nullable if needed - ActionsDAGPtr add_nullable_dag = changeTypesToNullable(cols_src, exclude_columns); - return add_nullable_dag; + return changeTypesToNullable(cols_src, exclude_columns); } void TableJoin::setStorageJoin(std::shared_ptr storage) @@ -957,7 +958,7 @@ bool TableJoin::allowParallelHashJoin() const return true; } -ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const +ActionsDAG TableJoin::createJoinedBlockActions(ContextPtr context) const { ASTPtr expression_list = rightKeysList(); auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable()); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 8e83233e54c..a057d46b94d 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -202,19 +202,19 @@ private: Names requiredJoinedNames() const; /// Create converting actions and change key column names if required - ActionsDAGPtr applyKeyConvertToTable( + std::optional applyKeyConvertToTable( const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, JoinTableSide table_side, NameToNameMap & key_column_rename); - ActionsDAGPtr applyNullsafeWrapper( + std::optional applyNullsafeWrapper( const ColumnsWithTypeAndName & cols_src, const NameSet & columns_for_nullsafe_comparison, JoinTableSide table_side, NameToNameMap & key_column_rename); - ActionsDAGPtr applyJoinUseNullsConversion( + std::optional applyJoinUseNullsConversion( const ColumnsWithTypeAndName & cols_src, const NameToNameMap & key_column_rename); @@ -264,7 +264,7 @@ public: TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } - ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const; + ActionsDAG createJoinedBlockActions(ContextPtr context) const; const std::vector & getEnabledJoinAlgorithms() const { return join_algorithm; } @@ -379,7 +379,7 @@ public: /// Calculate converting actions, rename key columns in required /// For `USING` join we will convert key columns inplace and affect into types in the result table /// For `JOIN ON` we will create new columns with converted keys to join by. - std::pair + std::pair, std::optional> createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns); diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index 929999c8c37..27d79e86622 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -14,15 +14,15 @@ namespace DB { -ActionsDAGPtr addMissingDefaults( +ActionsDAG addMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, ContextPtr context, bool null_as_default) { - auto actions = std::make_unique(header.getColumnsWithTypeAndName()); - auto & index = actions->getOutputs(); + ActionsDAG actions(header.getColumnsWithTypeAndName()); + auto & index = actions.getOutputs(); /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. /// First, remember the offset columns for all arrays in the block. @@ -40,7 +40,7 @@ ActionsDAGPtr addMissingDefaults( if (group.empty()) group.push_back(nullptr); - group.push_back(actions->getInputs()[i]); + group.push_back(actions.getInputs()[i]); } } @@ -62,11 +62,11 @@ ActionsDAGPtr addMissingDefaults( { const auto & nested_type = array_type->getNestedType(); ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0); - const auto & constant = actions->addColumn({nested_column, nested_type, column.name}); + const auto & constant = actions.addColumn({nested_column, nested_type, column.name}); auto & group = nested_groups[offsets_name]; group[0] = &constant; - index.push_back(&actions->addFunction(func_builder_replicate, group, constant.result_name)); + index.push_back(&actions.addFunction(func_builder_replicate, group, constant.result_name)); continue; } @@ -75,17 +75,17 @@ ActionsDAGPtr addMissingDefaults( * it can be full (or the interpreter may decide that it is constant everywhere). */ auto new_column = column.type->createColumnConstWithDefaultValue(0); - const auto * col = &actions->addColumn({new_column, column.type, column.name}); - index.push_back(&actions->materializeNode(*col)); + const auto * col = &actions.addColumn({new_column, column.type, column.name}); + index.push_back(&actions.materializeNode(*col)); } /// Computes explicitly specified values by default and materialized columns. - if (auto dag = evaluateMissingDefaults(actions->getResultColumns(), required_columns, columns, context, true, null_as_default)) - actions = ActionsDAG::merge(std::move(*actions), std::move(*dag)); + if (auto dag = evaluateMissingDefaults(actions.getResultColumns(), required_columns, columns, context, true, null_as_default)) + actions = ActionsDAG::merge(std::move(actions), std::move(*dag)); /// Removes unused columns and reorders result. - actions->removeUnusedActions(required_columns.getNames(), false); - actions->addMaterializingOutputActions(); + actions.removeUnusedActions(required_columns.getNames(), false); + actions.addMaterializingOutputActions(); return actions; } diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 94afd806dfd..5299bae9745 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -24,7 +24,7 @@ using ActionsDAGPtr = std::unique_ptr; * Also can substitute NULL with DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. * All three types of columns are materialized (not constants). */ -ActionsDAGPtr addMissingDefaults( +ActionsDAG addMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false); } diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 4e1a2bcf5ee..d5d9fce0dbd 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -89,7 +89,7 @@ std::optional evaluateConstantExpressionImpl(c ColumnPtr result_column; DataTypePtr result_type; String result_name = ast->getColumnName(); - for (const auto & action_node : actions->getOutputs()) + for (const auto & action_node : actions.getOutputs()) { if ((action_node->result_name == result_name) && action_node->column) { diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index b000264ae33..62f8aea86d1 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -152,22 +152,20 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi return conversion_expr_list; } -ActionsDAGPtr createExpressions( +std::optional createExpressions( const Block & header, ASTPtr expr_list, bool save_unneeded_columns, ContextPtr context) { if (!expr_list) - return nullptr; + return {}; auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context}; - auto dag = std::make_unique(header.getNamesAndTypesList()); + ActionsDAG dag(header.getNamesAndTypesList()); auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns); - dag = ActionsDAG::merge(std::move(*dag), std::move(*actions)); - - return dag; + return ActionsDAG::merge(std::move(dag), std::move(actions)); } } @@ -180,7 +178,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require if (auto dag = createExpressions(block, conversion_expr_list, true, context)) { - auto expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context)); + auto expression = std::make_shared(std::move(*dag), ExpressionActionsSettings::fromContext(context)); expression->execute(block); } } @@ -195,7 +193,7 @@ bool needConvertAnyNullToDefault(const Block & header, const NamesAndTypesList & return false; } -ActionsDAGPtr evaluateMissingDefaults( +std::optional evaluateMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, @@ -204,7 +202,7 @@ ActionsDAGPtr evaluateMissingDefaults( bool null_as_default) { if (!columns.hasDefaults() && (!null_as_default || !needConvertAnyNullToDefault(header, required_columns, columns))) - return nullptr; + return {}; ASTPtr expr_list = defaultRequiredExpressions(header, required_columns, columns, null_as_default); return createExpressions(header, expr_list, save_unneeded_columns, context); diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index ffc77561e79..570eb75dd4a 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -5,9 +5,6 @@ #include #include -#include -#include - namespace DB { @@ -24,12 +21,11 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; /// Create actions which adds missing defaults to block according to required_columns using columns description /// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. /// Return nullptr if no actions required. -ActionsDAGPtr evaluateMissingDefaults( +std::optional evaluateMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 16ee6de73c4..48e42099ce8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -332,12 +332,12 @@ void addExpressionStep(QueryPlan & query_plan, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = ActionsDAG::clone(&expression_actions->dag); + auto actions = std::move(*ActionsDAG::clone(&expression_actions->dag)); if (expression_actions->project_input) - actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); - appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets); + auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions)); + appendSetsFromActionsDAG(expression_step->getExpression(), useful_sets); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -347,15 +347,15 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); + auto actions = std::move(*ActionsDAG::clone(&filter_analysis_result.filter_actions->dag)); if (filter_analysis_result.filter_actions->project_input) - actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), - actions, + std::move(actions), filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); - appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets); + appendSetsFromActionsDAG(where_step->getExpression(), useful_sets); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -552,10 +552,10 @@ void addTotalsHavingStep(QueryPlan & query_plan, const auto & having_analysis_result = expression_analysis_result.getHaving(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); - ActionsDAGPtr actions; + std::optional actions; if (having_analysis_result.filter_actions) { - actions = ActionsDAG::clone(&having_analysis_result.filter_actions->dag); + actions = std::move(*ActionsDAG::clone(&having_analysis_result.filter_actions->dag)); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -564,7 +564,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, query_plan.getCurrentDataStream(), aggregation_analysis_result.aggregate_descriptions, query_analysis_result.aggregate_overflow_row, - actions, + std::move(actions), having_analysis_result.filter_column_name, having_analysis_result.remove_filter_column, settings.totals_mode, @@ -715,13 +715,13 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (query_node.hasInterpolate()) { - auto interpolate_actions_dag = std::make_unique(); + ActionsDAG interpolate_actions_dag; auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) { /// INTERPOLATE actions dag input columns must be non constant query_plan_column.column = nullptr; - interpolate_actions_dag->addInput(query_plan_column); + interpolate_actions_dag.addInput(query_plan_column); } auto & interpolate_list_node = query_node.getInterpolate()->as(); @@ -729,12 +729,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (interpolate_list_nodes.empty()) { - for (const auto * input_node : interpolate_actions_dag->getInputs()) + for (const auto * input_node : interpolate_actions_dag.getInputs()) { if (column_names_with_fill.contains(input_node->result_name)) continue; - interpolate_actions_dag->getOutputs().push_back(input_node); + interpolate_actions_dag.getOutputs().push_back(input_node); } } else @@ -744,12 +744,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, auto & interpolate_node_typed = interpolate_node->as(); PlannerActionsVisitor planner_actions_visitor(planner_context); - auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, + 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) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); - auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, + auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); if (interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); @@ -760,16 +760,16 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, const auto * interpolate_expression = interpolate_expression_nodes[0]; if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type)) { - interpolate_expression = &interpolate_actions_dag->addCast(*interpolate_expression, + interpolate_expression = &interpolate_actions_dag.addCast(*interpolate_expression, expression_to_interpolate->result_type, interpolate_expression->result_name); } - const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name); - interpolate_actions_dag->getOutputs().push_back(alias_node); + const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name); + interpolate_actions_dag.getOutputs().push_back(alias_node); } - interpolate_actions_dag->removeUnusedActions(); + interpolate_actions_dag.removeUnusedActions(); } Aliases empty_aliases; @@ -1130,7 +1130,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, return; auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - filter_info.actions, + std::move(*filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription("additional result filter"); @@ -1418,7 +1418,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(ActionsDAG::clone(filters.filter_actions)); + table_expression_data.setFilterActions(ActionsDAG::clone(&*filters.filter_actions)); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 59ec7778e21..4c0c9bc7937 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -757,12 +757,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } - auto lambda_actions_dag = std::make_unique(); - actions_stack.emplace_back(*lambda_actions_dag, node); + ActionsDAG lambda_actions_dag; + actions_stack.emplace_back(lambda_actions_dag, node); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); - lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); - lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); + lambda_actions_dag.getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag.removeUnusedActions(Names(1, lambda_expression_node_name)); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); @@ -879,14 +879,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & function_node = node->as(); auto function_node_name = action_node_name_helper.calculateActionNodeName(node); - auto index_hint_actions_dag = std::make_unique(); - auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); + ActionsDAG index_hint_actions_dag; + auto & index_hint_actions_dag_outputs = index_hint_actions_dag.getOutputs(); std::unordered_set index_hint_actions_dag_output_node_names; PlannerActionsVisitor actions_visitor(planner_context); for (const auto & argument : function_node.getArguments()) { - auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument); + auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument); for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes) { diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 418240fa34e..f35772ef7c0 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -25,7 +25,7 @@ class TableNode; struct FiltersForTableExpression { - ActionsDAGPtr filter_actions; + std::optional filter_actions; PrewhereInfoPtr prewhere_info; }; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 94054588d40..fa3a3483a8e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,19 +591,19 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) std::unique_ptr createComputeAliasColumnsStep( const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { - ActionsDAGPtr merged_alias_columns_actions_dag = std::make_unique(current_data_stream.header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); + ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs(); for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) { const auto & current_outputs = alias_column_actions_dag->getOutputs(); action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); - merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag)); + merged_alias_columns_actions_dag.mergeNodes(std::move(*alias_column_actions_dag)); } for (const auto * output_node : action_dag_outputs) - merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node); - merged_alias_columns_actions_dag->removeUnusedActions(false); + merged_alias_columns_actions_dag.addOrReplaceInOutputs(*output_node); + merged_alias_columns_actions_dag.removeUnusedActions(false); auto alias_column_step = std::make_unique(current_data_stream, std::move(merged_alias_columns_actions_dag)); alias_column_step->setStepDescription("Compute alias columns"); @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); + prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(prewhere_actions)); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -805,14 +805,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = std::move(filter_info.actions); + prewhere_info->prewhere_actions = std::move(*filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = std::move(filter_info.actions); + prewhere_info->row_level_filter = std::move(*filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } @@ -831,7 +831,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(row_policy_filter_info.actions)); + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&*row_policy_filter_info.actions)); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) @@ -964,15 +964,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan.addStep(std::move(alias_column_step)); } - for (const auto & filter_info_and_description : where_filters) + for (auto && [filter_info, description] : where_filters) { - const auto & [filter_info, description] = filter_info_and_description; if (query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns && filter_info.actions) { auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - filter_info.actions, + std::move(*filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription(description); @@ -1063,19 +1062,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (from_stage == QueryProcessingStage::FetchColumns) { - auto rename_actions_dag = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; - for (auto & output_node : rename_actions_dag->getOutputs()) + for (auto & output_node : rename_actions_dag.getOutputs()) { const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); if (!column_identifier) continue; - updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier)); + updated_actions_dag_outputs.push_back(&rename_actions_dag.addAlias(*output_node, *column_identifier)); } - rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); + rename_actions_dag.getOutputs() = std::move(updated_actions_dag_outputs); auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); rename_step->setStepDescription("Change column names to column identifiers"); @@ -1117,9 +1116,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { - auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (auto & output_node : cast_actions_dag->getOutputs()) + for (auto & output_node : cast_actions_dag.getOutputs()) { if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) { @@ -1128,11 +1127,11 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP type_to_check = type_to_check_low_cardinality->getDictionaryType(); if (type_to_check->canBeInsideNullable()) - output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + output_node = &cast_actions_dag.addFunction(to_nullable_function, {output_node}, output_node->result_name); } } - cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); + cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); plan_to_add_cast.addStep(std::move(cast_join_columns_step)); @@ -1178,16 +1177,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); - join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); - auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); + join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); + auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions)); left_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); left_plan.addStep(std::move(left_join_expressions_actions_step)); - join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); - auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); + join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); + auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions)); right_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1225,19 +1224,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) { - auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (auto & output_node : cast_actions_dag->getOutputs()) + for (auto & output_node : cast_actions_dag.getOutputs()) { auto it = plan_column_name_to_cast_type.find(output_node->result_name); if (it == plan_column_name_to_cast_type.end()) continue; const auto & cast_type = it->second; - output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name); + output_node = &cast_actions_dag.addCast(*output_node, cast_type, output_node->result_name); } - cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); + cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); @@ -1385,7 +1384,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ { ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( - std::move(join_clauses_and_actions.mixed_join_expressions_actions), + std::move(*join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); @@ -1542,12 +1541,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - auto drop_unused_columns_after_join_actions_dag = std::make_unique(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; std::optional first_skipped_column_node_index; - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag->getOutputs(); + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) @@ -1619,7 +1618,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto plan = std::move(join_tree_query_plan.query_plan); auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr array_join_action_dag = std::make_unique(plan_output_columns); + ActionsDAG array_join_action_dag(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); std::unordered_set array_join_expressions_output_nodes; @@ -1630,28 +1629,28 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_column_names.insert(array_join_column_identifier); auto & array_join_expression_column = array_join_expression->as(); - auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); + auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); for (auto & expression_dag_index_node : expression_dag_index_nodes) { - const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier); - array_join_action_dag->getOutputs().push_back(array_join_column_node); + const auto * array_join_column_node = &array_join_action_dag.addAlias(*expression_dag_index_node, array_join_column_identifier); + array_join_action_dag.getOutputs().push_back(array_join_column_node); array_join_expressions_output_nodes.insert(array_join_column_node->result_name); } } - array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + array_join_action_dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); - appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(array_join_actions->getExpression(), join_tree_query_plan.useful_sets); plan.addStep(std::move(array_join_actions)); - auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG drop_unused_columns_before_array_join_actions_dag(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_before_array_join_actions_dag_updated_outputs_names; - auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag->getOutputs(); + auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag.getOutputs(); size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size(); for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 23b6a805ab9..db9678d91a6 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -177,13 +177,13 @@ std::set extractJoinTableSidesFromExpression(//const ActionsDAG:: } const ActionsDAG::Node * appendExpression( - ActionsDAGPtr & dag, + ActionsDAG & dag, const QueryTreeNodePtr & expression, const PlannerContextPtr & planner_context, const JoinNode & join_node) { PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", @@ -193,9 +193,9 @@ const ActionsDAG::Node * appendExpression( } void buildJoinClause( - ActionsDAGPtr & left_dag, - ActionsDAGPtr & right_dag, - ActionsDAGPtr & mixed_dag, + ActionsDAG & left_dag, + ActionsDAG & right_dag, + ActionsDAG & mixed_dag, const PlannerContextPtr & planner_context, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, @@ -376,8 +376,8 @@ JoinClausesAndActions buildJoinClausesAndActions( const JoinNode & join_node, const PlannerContextPtr & planner_context) { - ActionsDAGPtr left_join_actions = std::make_unique(left_table_expression_columns); - ActionsDAGPtr right_join_actions = std::make_unique(right_table_expression_columns); + ActionsDAG left_join_actions(left_table_expression_columns); + ActionsDAG right_join_actions(right_table_expression_columns); ColumnsWithTypeAndName mixed_table_expression_columns; for (const auto & left_column : left_table_expression_columns) { @@ -387,7 +387,7 @@ JoinClausesAndActions buildJoinClausesAndActions( { mixed_table_expression_columns.push_back(right_column); } - ActionsDAGPtr mixed_join_actions = std::make_unique(mixed_table_expression_columns); + ActionsDAG mixed_join_actions(mixed_table_expression_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -498,12 +498,12 @@ JoinClausesAndActions buildJoinClausesAndActions( { const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (left_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {}); + dag_filter_condition_node = &left_join_actions.addFunction(and_function, left_filter_condition_nodes, {}); else dag_filter_condition_node = left_filter_condition_nodes[0]; join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node}; - left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + left_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); } @@ -514,12 +514,12 @@ JoinClausesAndActions buildJoinClausesAndActions( const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (right_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {}); + dag_filter_condition_node = &right_join_actions.addFunction(and_function, right_filter_condition_nodes, {}); else dag_filter_condition_node = right_filter_condition_nodes[0]; join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node}; - right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + right_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); } @@ -556,10 +556,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } if (!left_key_node->result_type->equals(*common_type)) - left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); + left_key_node = &left_join_actions.addCast(*left_key_node, common_type, {}); if (!right_key_node->result_type->equals(*common_type)) - right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); + right_key_node = &right_join_actions.addCast(*right_key_node, common_type, {}); } if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable()) @@ -576,24 +576,24 @@ JoinClausesAndActions buildJoinClausesAndActions( * SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b) */ auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext()); - left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {}); - right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {}); + left_key_node = &left_join_actions.addFunction(wrap_nullsafe_function, {left_key_node}, {}); + right_key_node = &right_join_actions.addFunction(wrap_nullsafe_function, {right_key_node}, {}); } - left_join_actions->addOrReplaceInOutputs(*left_key_node); - right_join_actions->addOrReplaceInOutputs(*right_key_node); + left_join_actions.addOrReplaceInOutputs(*left_key_node); + right_join_actions.addOrReplaceInOutputs(*right_key_node); add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); } } - result.left_join_expressions_actions = ActionsDAG::clone(left_join_actions); - result.left_join_tmp_expression_actions = std::move(left_join_actions); - result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = ActionsDAG::clone(right_join_actions); - result.right_join_tmp_expression_actions = std::move(right_join_actions); - result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + result.left_join_expressions_actions = std::move(left_join_actions); + //result.left_join_tmp_expression_actions = std::move(left_join_actions); + result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); + result.right_join_expressions_actions = std::move(right_join_actions); + //result.right_join_tmp_expression_actions = std::move(right_join_actions); + result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); if (is_inequal_join) { @@ -601,16 +601,16 @@ JoinClausesAndActions buildJoinClausesAndActions( /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - auto mixed_join_expressions_actions = std::make_unique(mixed_table_expression_columns); + ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception( ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); - mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); + mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; - mixed_join_expressions_actions->removeUnusedActions(required_names); + mixed_join_expressions_actions.removeUnusedActions(required_names); result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } else diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 8adf6edd7ea..3735c373acc 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -182,15 +182,15 @@ struct JoinClausesAndActions /// Join clauses. Actions dag nodes point into join_expression_actions. JoinClauses join_clauses; /// Whole JOIN ON section expressions - ActionsDAGPtr left_join_tmp_expression_actions; - ActionsDAGPtr right_join_tmp_expression_actions; + // ActionsDAGPtr left_join_tmp_expression_actions; + // ActionsDAGPtr right_join_tmp_expression_actions; /// Left join expressions actions - ActionsDAGPtr left_join_expressions_actions; + ActionsDAG left_join_expressions_actions; /// Right join expressions actions - ActionsDAGPtr right_join_expressions_actions; + ActionsDAG right_join_expressions_actions; /// Originally used for inequal join. it's the total join expression. /// If there is no inequal join conditions, it's null. - ActionsDAGPtr mixed_join_expressions_actions; + std::optional mixed_join_expressions_actions; }; /** Calculate join clauses and actions for JOIN ON section. diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 7ac53e0f8c1..e9f9c51d338 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -442,22 +442,22 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSets(filter_query_tree, *planner_context); - auto filter_actions_dag = std::make_unique(); + ActionsDAG filter_actions_dag; PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); + auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filter actions must return single output node. Actual {}", expression_nodes.size()); - auto & filter_actions_outputs = filter_actions_dag->getOutputs(); + auto & filter_actions_outputs = filter_actions_dag.getOutputs(); filter_actions_outputs = std::move(expression_nodes); std::string filter_node_name = filter_actions_outputs[0]->result_name; bool remove_filter_column = true; - for (const auto & filter_input_node : filter_actions_dag->getInputs()) + for (const auto & filter_input_node : filter_actions_dag.getInputs()) if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) filter_actions_outputs.push_back(filter_input_node); @@ -498,7 +498,7 @@ void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets); + appendSetsFromActionsDAG(index_hint->getActions(), useful_sets); } } } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 64ba7f7cd2a..f31de80b22d 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -303,15 +303,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B const auto & header = ports[set_counter]->getHeader(); /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - auto dag = std::make_unique(header.getColumnsWithTypeAndName()); + ActionsDAG dag(header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(output_header.columns() + 1); auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0); - const auto * grouping_node = &dag->addColumn( + const auto * grouping_node = &dag.addColumn( {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - grouping_node = &dag->materializeNode(*grouping_node); + grouping_node = &dag.materializeNode(*grouping_node); outputs.push_back(grouping_node); const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; @@ -332,21 +332,21 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B column_with_default->finalize(); auto column = ColumnConst::create(std::move(column_with_default), 0); - const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name}); - node = &dag->materializeNode(*node); + const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); + node = &dag.materializeNode(*node); outputs.push_back(node); } else { - const auto * column_node = dag->getOutputs()[header.getPositionByName(col.name)]; + const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)]; if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) - outputs.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name)); + outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); else outputs.push_back(column_node); } } - dag->getOutputs().swap(outputs); + dag.getOutputs().swap(outputs); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index b6c70061987..3a98f8e4612 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -36,27 +36,27 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { - auto dag = std::make_unique(header.getColumnsWithTypeAndName()); - auto & outputs = dag->getOutputs(); + ActionsDAG dag(header.getColumnsWithTypeAndName()); + auto & outputs = dag.getOutputs(); if (use_nulls) { auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); for (const auto & key : keys) { - const auto * node = dag->getOutputs()[header.getPositionByName(key)]; + const auto * node = dag.getOutputs()[header.getPositionByName(key)]; if (node->result_type->canBeInsideNullable()) { - dag->addOrReplaceInOutputs(dag->addFunction(to_nullable, { node }, node->result_name)); + dag.addOrReplaceInOutputs(dag.addFunction(to_nullable, { node }, node->result_name)); } } } auto grouping_col = ColumnUInt64::create(1, grouping_set_number); - const auto * grouping_node = &dag->addColumn( + const auto * grouping_node = &dag.addColumn( {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - grouping_node = &dag->materializeNode(*grouping_node); + grouping_node = &dag.materializeNode(*grouping_node); outputs.insert(outputs.begin(), grouping_node); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 1f4f271fa6e..1c199ebedb3 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -32,7 +32,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi }; auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); plan.addStep(std::move(converting)); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 50bc2e1533e..94098f443d9 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -10,33 +10,33 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions, const Block & header, const SortDescription & sort_description) +static ITransformingStep::Traits getTraits(const ActionsDAG & actions, const Block & header, const SortDescription & sort_description) { return ITransformingStep::Traits { { .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = actions->isSortingPreserved(header, sort_description), + .preserves_sorting = actions.isSortingPreserved(header, sort_description), }, { - .preserves_number_of_rows = !actions->hasArrayJoin(), + .preserves_number_of_rows = !actions.hasArrayJoin(), } }; } -ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_) +ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_) : ITransformingStep( input_stream_, - ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), + ExpressionTransform::transformHeader(input_stream_.header, actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(ActionsDAG::clone(actions_dag_)) + , actions_dag(std::move(actions_dag_)) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()); + auto expression = std::make_shared(std::move(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,25 +61,25 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); map.add("Expression", expression->toTree()); } void ExpressionStep::updateOutputStream() { output_stream = createOutputStream( - input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits()); + input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, actions_dag), getDataStreamTraits()); if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(*actions_dag); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index ebbac8217cb..f2926318cbc 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -1,12 +1,10 @@ #pragma once #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class ExpressionTransform; class JoiningTransform; @@ -15,21 +13,22 @@ class ExpressionStep : public ITransformingStep { public: - explicit ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_); + explicit ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_); String getName() const override { return "Expression"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions_dag; } + ActionsDAG & getExpression() { return actions_dag; } + const ActionsDAG & getExpression() const { return actions_dag; } void describeActions(JSONBuilder::JSONMap & map) const override; private: void updateOutputStream() override; - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; }; } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 7883461f45a..5f15c5defac 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -9,9 +9,9 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) +static ITransformingStep::Traits getTraits(const ActionsDAG & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) { - bool preserves_sorting = expression->isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); + bool preserves_sorting = expression.isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); if (remove_filter_column) { preserves_sorting &= std::find_if( @@ -35,22 +35,22 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, con FilterStep::FilterStep( const DataStream & input_stream_, - const ActionsDAGPtr & actions_dag_, + ActionsDAG actions_dag_, String filter_column_name_, bool remove_filter_column_) : ITransformingStep( input_stream_, FilterTransform::transformHeader( input_stream_.header, - actions_dag_.get(), + &actions_dag_, filter_column_name_, remove_filter_column_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_)) + , actions_dag(std::move(actions_dag_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { - actions_dag = ActionsDAG::clone(actions_dag_); - actions_dag->removeAliasesForFilter(filter_column_name); + actions_dag.removeAliasesForFilter(filter_column_name); } void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); map.add("Expression", expression->toTree()); } @@ -104,13 +104,13 @@ void FilterStep::updateOutputStream() { output_stream = createOutputStream( input_streams.front(), - FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column), + FilterTransform::transformHeader(input_streams.front().header, &actions_dag, filter_column_name, remove_filter_column), getDataStreamTraits()); if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(*actions_dag); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index 0f894a570b7..b5a31bef5fc 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -1,19 +1,17 @@ #pragma once #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - /// Implements WHERE, HAVING operations. See FilterTransform. class FilterStep : public ITransformingStep { public: FilterStep( const DataStream & input_stream_, - const ActionsDAGPtr & actions_dag_, + ActionsDAG actions_dag_, String filter_column_name_, bool remove_filter_column_); @@ -23,15 +21,15 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions_dag; } - ActionsDAGPtr & getExpression() { return actions_dag; } + const ActionsDAG & getExpression() const { return actions_dag; } + ActionsDAG & getExpression() { return actions_dag; } const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } private: void updateOutputStream() override; - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; String filter_column_name; bool remove_filter_column; }; diff --git a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp index d90f0e152e7..be468419cfb 100644 --- a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp @@ -45,10 +45,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan:: bool right_stream_safe = true; if (check_left_stream) - left_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header); + left_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header); if (check_right_stream) - right_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header); + right_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header); if (!left_stream_safe || !right_stream_safe) return 0; diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 6cdc3cb4eb0..8666912514e 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) steps_to_update.push_back(step); if (const auto * const expr = typeid_cast(step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index f26cd79dd97..411b20b1a32 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -101,7 +101,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node) return res; } -static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) +static std::optional splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) { QueryPlan::Node * child_node = parent_node->children.front(); checkChildrenSize(child_node, child_idx + 1); @@ -110,16 +110,16 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & av auto & child = child_node->step; auto * filter = assert_cast(parent.get()); - const auto & expression = filter->getExpression(); + auto & expression = filter->getExpression(); const auto & filter_column_name = filter->getFilterColumnName(); bool removes_filter = filter->removesFilterColumn(); const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); - return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); + return expression.splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); } static size_t -addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter, +addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, ActionsDAG split_filter, bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true) { QueryPlan::Node * child_node = parent_node->children.front(); @@ -129,14 +129,14 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, auto & child = child_node->step; auto * filter = assert_cast(parent.get()); - const auto & expression = filter->getExpression(); + auto & expression = filter->getExpression(); const auto & filter_column_name = filter->getFilterColumnName(); - const auto * filter_node = expression->tryFindInOutputs(filter_column_name); + const auto * filter_node = expression.tryFindInOutputs(filter_column_name); if (update_parent_filter && !filter_node && !filter->removesFilterColumn()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, expression->dumpDAG()); + filter_column_name, expression.dumpDAG()); /// Add new Filter step before Child. /// Expression/Filter -> Child -> Something @@ -147,10 +147,10 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, /// Expression/Filter -> Child -> Filter -> Something /// New filter column is the first one. - String split_filter_column_name = split_filter->getOutputs().front()->result_name; + String split_filter_column_name = split_filter.getOutputs().front()->result_name; node.step = std::make_unique( - node.children.at(0)->step->getOutputStream(), split_filter, std::move(split_filter_column_name), can_remove_filter); + node.children.at(0)->step->getOutputStream(), std::move(split_filter), std::move(split_filter_column_name), can_remove_filter); if (auto * transforming_step = dynamic_cast(child.get())) { @@ -176,7 +176,7 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, { /// This means that all predicates of filter were pushed down. /// Replace current actions to expression, as we don't need to filter anything. - parent = std::make_unique(child->getOutputStream(), expression); + parent = std::make_unique(child->getOutputStream(), std::move(expression)); } else { @@ -192,7 +192,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con bool can_remove_filter = true, size_t child_idx = 0) { if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) - return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx); + return addNewFilterStepOrThrow(parent_node, nodes, std::move(*split_filter), can_remove_filter, child_idx); return 0; } @@ -332,7 +332,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available); Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available); - auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), + auto join_filter_push_down_actions = filter->getExpression().splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), filter->removesFilterColumn(), left_stream_available_columns_to_push_down, left_stream_input_header, @@ -346,42 +346,44 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: if (join_filter_push_down_actions.left_stream_filter_to_push_down) { + const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name; updated_steps += addNewFilterStepOrThrow(parent_node, nodes, - join_filter_push_down_actions.left_stream_filter_to_push_down, + std::move(*join_filter_push_down_actions.left_stream_filter_to_push_down), join_filter_push_down_actions.left_stream_filter_removes_filter, 0 /*child_idx*/, false /*update_parent_filter*/); LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", - join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name, + result_name, JoinKind::Left); } if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right) { + const auto & result_name = join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name; updated_steps += addNewFilterStepOrThrow(parent_node, nodes, - join_filter_push_down_actions.right_stream_filter_to_push_down, + std::move(*join_filter_push_down_actions.right_stream_filter_to_push_down), join_filter_push_down_actions.right_stream_filter_removes_filter, 1 /*child_idx*/, false /*update_parent_filter*/); LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", - join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name, + result_name, JoinKind::Right); } if (updated_steps > 0) { const auto & filter_column_name = filter->getFilterColumnName(); - const auto & filter_expression = filter->getExpression(); + auto & filter_expression = filter->getExpression(); - const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name); + const auto * filter_node = filter_expression.tryFindInOutputs(filter_column_name); if (!filter_node && !filter->removesFilterColumn()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, filter_expression->dumpDAG()); + filter_column_name, filter_expression.dumpDAG()); /// Filter column was replaced to constant. @@ -391,7 +393,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: { /// This means that all predicates of filter were pushed down. /// Replace current actions to expression, as we don't need to filter anything. - parent = std::make_unique(child->getOutputStream(), filter_expression); + parent = std::make_unique(child->getOutputStream(), std::move(filter_expression)); } else { @@ -416,7 +418,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (!filter) return 0; - if (filter->getExpression()->hasStatefulFunctions()) + if (filter->getExpression().hasStatefulFunctions()) return 0; if (auto * aggregating = typeid_cast(child.get())) @@ -430,7 +432,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 0; const auto & actions = filter->getExpression(); - const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName()); + const auto & filter_node = actions.findInOutputs(filter->getFilterColumnName()); auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node); @@ -597,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - ActionsDAG::clone(filter->getExpression()), + std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -611,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp index 36aab41df49..0d4f2330119 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp @@ -28,10 +28,10 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node const auto & expression = expression_step ? expression_step->getExpression() : filter_step->getExpression(); - auto split_actions = expression->splitActionsBeforeArrayJoin(array_join->columns); + auto split_actions = expression.splitActionsBeforeArrayJoin(array_join->columns); /// No actions can be moved before ARRAY JOIN. - if (split_actions.first->trivial()) + if (split_actions.first.trivial()) return 0; auto description = parent->getStepDescription(); @@ -49,9 +49,9 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node array_join_step->updateInputStream(node.step->getOutputStream()); if (expression_step) - parent = std::make_unique(array_join_step->getOutputStream(), split_actions.second); + parent = std::make_unique(array_join_step->getOutputStream(), std::move(split_actions.second)); else - parent = std::make_unique(array_join_step->getOutputStream(), split_actions.second, + parent = std::make_unique(array_join_step->getOutputStream(), std::move(split_actions.second), filter_step->getFilterColumnName(), filter_step->removesFilterColumn()); parent->setStepDescription(description + " [split]"); diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index b280e2d3cc6..7794ddae8fa 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -66,13 +66,13 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: NameSet sort_columns; for (const auto & col : sorting_step->getSortDescription()) sort_columns.insert(col.column_name); - auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns); + auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression().splitActionsBySortingDescription(sort_columns); // No calculations can be postponed. - if (unneeded_for_sorting->trivial()) + if (unneeded_for_sorting.trivial()) return 0; - if (!areNodesConvertableToBlock(needed_for_sorting->getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting->getInputs())) + if (!areNodesConvertableToBlock(needed_for_sorting.getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting.getInputs())) return 0; // Sorting (parent_node) -> Expression (child_node) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 4629bc0af53..53f59198d0f 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - ActionsDAG::clone(expression->getExpression())); + std::move(*ActionsDAG::clone(&expression->getExpression()))); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 97de69b1134..d7ca96e4c64 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -38,18 +38,18 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (parent_expr && child_expr) { - const auto & child_actions = child_expr->getExpression(); - const auto & parent_actions = parent_expr->getExpression(); + auto & child_actions = child_expr->getExpression(); + auto & parent_actions = parent_expr->getExpression(); /// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them. /// Example: select rowNumberInBlock() from (select arrayJoin([1, 2])) /// Such a query will return two zeroes if we combine actions together. - if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions()) return 0; - auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); + auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions)); - auto expr = std::make_unique(child_expr->getInputStreams().front(), merged); + auto expr = std::make_unique(child_expr->getInputStreams().front(), std::move(merged)); expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); parent_node->step = std::move(expr); @@ -58,16 +58,16 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) } else if (parent_filter && child_expr) { - const auto & child_actions = child_expr->getExpression(); - const auto & parent_actions = parent_filter->getExpression(); + auto & child_actions = child_expr->getExpression(); + auto & parent_actions = parent_filter->getExpression(); - if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions()) return 0; - auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); + auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions)); auto filter = std::make_unique(child_expr->getInputStreams().front(), - merged, + std::move(merged), parent_filter->getFilterColumnName(), parent_filter->removesFilterColumn()); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); @@ -78,32 +78,31 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) } else if (parent_filter && child_filter) { - const auto & child_actions = child_filter->getExpression(); - const auto & parent_actions = parent_filter->getExpression(); + auto & child_actions = child_filter->getExpression(); + auto & parent_actions = parent_filter->getExpression(); - if (child_actions->hasArrayJoin()) + if (child_actions.hasArrayJoin()) return 0; - auto actions = ActionsDAG::clone(child_actions); - const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName()); + const auto & child_filter_node = child_actions.findInOutputs(child_filter->getFilterColumnName()); if (child_filter->removesFilterColumn()) - removeFromOutputs(*actions, child_filter_node); + removeFromOutputs(child_actions, child_filter_node); - actions->mergeInplace(std::move(*ActionsDAG::clone(parent_actions))); + child_actions.mergeInplace(std::move(parent_actions)); - const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); + const auto & parent_filter_node = child_actions.findInOutputs(parent_filter->getFilterColumnName()); if (parent_filter->removesFilterColumn()) - removeFromOutputs(*actions, parent_filter_node); + removeFromOutputs(child_actions, parent_filter_node); FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto & condition = actions->addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); - auto & outputs = actions->getOutputs(); + const auto & condition = child_actions.addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); + auto & outputs = child_actions.getOutputs(); outputs.insert(outputs.begin(), &condition); - actions->removeUnusedActions(false); + child_actions.removeUnusedActions(false); auto filter = std::make_unique(child_filter->getInputStreams().front(), - actions, + std::move(child_actions), condition.result_name, true); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")"); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index f203d831750..0d9e050d6cb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -83,10 +83,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) Names queried_columns = source_step_with_filter->requiredSourceColumns(); + const auto & source_filter_actions_dag = source_step_with_filter->getFilterActionsDAG(); MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), storage_metadata, - storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context), + storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_filter_actions_dag ? &*source_filter_actions_dag : nullptr, context), queried_columns, storage.supportedPrewhereColumns(), getLogger("QueryPlanOptimizePrewhere")}; @@ -113,15 +114,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (prewhere_info->remove_prewhere_column) { - removeFromOutput(*filter_expression, filter_column_name); - auto & outputs = filter_expression->getOutputs(); + removeFromOutput(filter_expression, filter_column_name); + auto & outputs = filter_expression.getOutputs(); size_t size = outputs.size(); outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end()); - filter_expression->removeUnusedActions(false); + filter_expression.removeUnusedActions(false); outputs.resize(size); } - auto split_result = filter_expression->split(optimize_result.prewhere_nodes, true, true); + auto split_result = filter_expression.split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. @@ -137,15 +138,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// So, here we restore removed inputs for PREWHERE actions { std::unordered_set first_outputs( - split_result.first->getOutputs().begin(), split_result.first->getOutputs().end()); - for (const auto * input : split_result.first->getInputs()) + split_result.first.getOutputs().begin(), split_result.first.getOutputs().end()); + for (const auto * input : split_result.first.getInputs()) { if (!first_outputs.contains(input)) { - split_result.first->getOutputs().push_back(input); + split_result.first.getOutputs().push_back(input); /// Add column to second actions as input. /// Do not add it to result, so it would be removed. - split_result.second->addInput(input->result_name, input->result_type); + split_result.second.addInput(input->result_name, input->result_type); } } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 0afddede708..71a7ca327b1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(ActionsDAG::clone(filter_step->getExpression()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(ActionsDAG::clone(&filter_step->getExpression()), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index a8bd98d7460..b3747b81215 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -170,12 +170,12 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt } } -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); + dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); else - dag = ActionsDAG::clone(expression); + dag = ActionsDAG::clone(&expression); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. @@ -193,7 +193,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & if (prewhere_info->prewhere_actions) { //std::cerr << "====== Adding prewhere " << std::endl; - appendExpression(dag, prewhere_info->prewhere_actions); + appendExpression(dag, *prewhere_info->prewhere_actions); if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); } @@ -211,7 +211,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & const auto & actions = expression->getExpression(); /// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array. - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) limit = 0; appendExpression(dag, actions); @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index da057bd25c2..34e9c8aac0e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -273,7 +273,7 @@ static void appendAggregateFunctions( } } -ActionsDAGPtr analyzeAggregateProjection( +std::optional analyzeAggregateProjection( const AggregateProjectionInfo & info, const QueryDAG & query, const DAGIndex & query_index, @@ -393,7 +393,7 @@ ActionsDAGPtr analyzeAggregateProjection( // LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection"); auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes); - appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates); + appendAggregateFunctions(proj_dag, aggregates, *matched_aggregates); return proj_dag; } @@ -405,7 +405,7 @@ struct AggregateProjectionCandidate : public ProjectionCandidate /// Actions which need to be applied to columns from projection /// in order to get all the columns required for aggregation. - ActionsDAGPtr dag; + ActionsDAG dag; }; struct MinMaxProjectionCandidate @@ -480,13 +480,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)}; // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, - candidate.dag->getRequiredColumnsNames(), - (dag.filter_node ? dag.dag.get() : nullptr), + candidate.dag.getRequiredColumnsNames(), + (dag.filter_node ? &*dag.dag : nullptr), parts, max_added_blocks.get(), context); @@ -536,7 +536,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)}; candidate.projection = projection; candidates.real.emplace_back(std::move(candidate)); } @@ -664,7 +664,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu /// Selecting best candidate. for (auto & candidate : candidates.real) { - auto required_column_names = candidate.dag->getRequiredColumnsNames(); + auto required_column_names = candidate.dag.getRequiredColumnsNames(); bool analyzed = analyzeProjectionCandidate( candidate, @@ -675,7 +675,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu query_info, context, max_added_blocks, - candidate.dag.get()); + &candidate.dag); if (!analyzed) continue; @@ -765,7 +765,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = reader.readFromParts( /* parts = */ {}, /* alter_conversions = */ {}, - best_candidate->dag->getRequiredColumnsNames(), + best_candidate->dag.getRequiredColumnsNames(), proj_snapshot, projection_query_info, context, @@ -777,7 +777,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu if (!projection_reading) { - auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); + auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag.getRequiredColumnsNames()); Pipe pipe(std::make_shared(std::move(header))); projection_reading = std::make_unique(std::move(pipe)); } @@ -808,17 +808,19 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu if (best_candidate) { aggregate_projection_node = &nodes.emplace_back(); + if (candidates.has_filter) { + const auto & result_name = best_candidate->dag.getOutputs().front()->result_name; aggregate_projection_node->step = std::make_unique( projection_reading_node.step->getOutputStream(), - best_candidate->dag, - best_candidate->dag->getOutputs().front()->result_name, + std::move(best_candidate->dag), + result_name, true); } else aggregate_projection_node->step - = std::make_unique(projection_reading_node.step->getOutputStream(), best_candidate->dag); + = std::make_unique(projection_reading_node.step->getOutputStream(), std::move(best_candidate->dag)); aggregate_projection_node->children.push_back(&projection_reading_node); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index c7e96d66817..c0af178f08e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -23,7 +23,7 @@ struct NormalProjectionCandidate : public ProjectionCandidate { }; -static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_header) +static std::optional makeMaterializingDAG(const Block & proj_header, const Block main_header) { /// Materialize constants in case we don't have it in output header. /// This may happen e.g. if we have PREWHERE. @@ -31,7 +31,7 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block size_t num_columns = main_header.columns(); /// This is a error; will have block structure mismatch later. if (proj_header.columns() != num_columns) - return nullptr; + return {}; std::vector const_positions; for (size_t i = 0; i < num_columns; ++i) @@ -45,17 +45,17 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block } if (const_positions.empty()) - return nullptr; + return {}; - ActionsDAGPtr dag = std::make_unique(); - auto & outputs = dag->getOutputs(); + ActionsDAG dag; + auto & outputs = dag.getOutputs(); for (const auto & col : proj_header.getColumnsWithTypeAndName()) - outputs.push_back(&dag->addInput(col)); + outputs.push_back(&dag.addInput(col)); for (auto pos : const_positions) { auto & output = outputs[pos]; - output = &dag->materializeNode(*output); + output = &dag.materializeNode(*output); } return dag; @@ -172,7 +172,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod query_info, context, max_added_blocks, - query.filter_node ? query.dag.get() : nullptr); + query.filter_node ? &*query.dag : nullptr); if (!analyzed) continue; @@ -242,14 +242,14 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod { expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - query.dag, + std::move(*query.dag), query.filter_node->result_name, true); } else expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - query.dag); + std::move(*query.dag)); expr_or_filter_node.children.push_back(&projection_reading_node); next_node = &expr_or_filter_node; @@ -267,7 +267,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) { - auto converting = std::make_unique(*proj_stream, materializing); + auto converting = std::make_unique(*proj_stream, std::move(*materializing)); proj_stream = &converting->getOutputStream(); auto & expr_node = nodes.emplace_back(); expr_node.step = std::move(converting); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 0e2ad96a419..fb2e6c2096e 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -64,12 +64,12 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea return {}; } -void QueryDAG::appendExpression(const ActionsDAGPtr & expression) +void QueryDAG::appendExpression(const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); + dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); else - dag = ActionsDAG::clone(expression); + dag = std::move(*ActionsDAG::clone(&expression)); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -120,7 +120,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & { if (prewhere_info->row_level_filter) { - appendExpression(prewhere_info->row_level_filter); + appendExpression(*prewhere_info->row_level_filter); if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false)) filter_nodes.push_back(filter_expression); else @@ -129,7 +129,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (prewhere_info->prewhere_actions) { - appendExpression(prewhere_info->prewhere_actions); + appendExpression(*prewhere_info->prewhere_actions); if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) filter_nodes.push_back(filter_expression); @@ -149,7 +149,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (auto * expression = typeid_cast(step)) { const auto & actions = expression->getExpression(); - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) return false; appendExpression(actions); @@ -159,7 +159,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (auto * filter = typeid_cast(step)) { const auto & actions = filter->getExpression(); - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) return false; appendExpression(actions); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 59ad3a43b97..ee0dfddc326 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -25,14 +25,14 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. struct QueryDAG { - ActionsDAGPtr dag; + std::optional dag; const ActionsDAG::Node * filter_node = nullptr; bool build(QueryPlan::Node & node); private: bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes); - void appendExpression(const ActionsDAGPtr & expression); + void appendExpression(const ActionsDAG & expression); }; struct ProjectionCandidate diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 81a8a537830..d0acd8221d4 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -132,10 +132,10 @@ namespace return true; if (const auto * const expr = typeid_cast(step); expr) - return !expr->getExpression()->hasArrayJoin(); + return !expr->getExpression().hasArrayJoin(); if (const auto * const filter = typeid_cast(step); filter) - return !filter->getExpression()->hasArrayJoin(); + return !filter->getExpression().hasArrayJoin(); if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) @@ -183,9 +183,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); if (inner_distinct_step = typeid_cast(node->step.get()); inner_distinct_step) @@ -236,9 +236,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); inner_distinct_step = typeid_cast(node->step.get()); diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 632eba6ab5f..7cac7bee6ec 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -213,12 +213,12 @@ private: logStep("checking for stateful function", node); if (const auto * expr = typeid_cast(step); expr) { - if (expr->getExpression()->hasStatefulFunctions()) + if (expr->getExpression().hasStatefulFunctions()) return false; } else if (const auto * filter = typeid_cast(step); filter) { - if (filter->getExpression()->hasStatefulFunctions()) + if (filter->getExpression().hasStatefulFunctions()) return false; } else diff --git a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp index 561ad7302c6..6aed57634b0 100644 --- a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp +++ b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp @@ -17,13 +17,13 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) const std::string & filter_column_name = filter_step->getFilterColumnName(); /// Do not split if there are function like runningDifference. - if (expr->hasStatefulFunctions()) + if (expr.hasStatefulFunctions()) return 0; bool filter_name_clashs_with_input = false; if (filter_step->removesFilterColumn()) { - for (const auto * input : expr->getInputs()) + for (const auto * input : expr.getInputs()) { if (input->result_name == filter_column_name) { @@ -33,14 +33,14 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) } } - auto split = expr->splitActionsForFilter(filter_column_name); + auto split = expr.splitActionsForFilter(filter_column_name); - if (split.second->trivial()) + if (split.second.trivial()) return 0; bool remove_filter = false; if (filter_step->removesFilterColumn()) - remove_filter = split.second->removeUnusedResult(filter_column_name); + remove_filter = split.second.removeUnusedResult(filter_column_name); auto description = filter_step->getStepDescription(); @@ -53,11 +53,11 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) { split_filter_name = "__split_filter"; - for (auto & filter_output : split.first->getOutputs()) + for (auto & filter_output : split.first.getOutputs()) { if (filter_output->result_name == filter_column_name) { - filter_output = &split.first->addAlias(*filter_output, split_filter_name); + filter_output = &split.first.addAlias(*filter_output, split_filter_name); break; } } diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 124cb735d5a..7e0260c0040 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -146,16 +146,16 @@ bool allOutputsDependsOnlyOnAllowedNodes( /// 3. We match partition key actions with group by key actions to find col1', ..., coln' in partition key actions. /// 4. We check that partition key is indeed a deterministic function of col1', ..., coln'. bool isPartitionKeySuitsGroupByKey( - const ReadFromMergeTree & reading, const ActionsDAGPtr & group_by_actions, const AggregatingStep & aggregating) + const ReadFromMergeTree & reading, const ActionsDAG & group_by_actions, const AggregatingStep & aggregating) { if (aggregating.isGroupingSets()) return false; - if (group_by_actions->hasArrayJoin() || group_by_actions->hasStatefulFunctions() || group_by_actions->hasNonDeterministic()) + if (group_by_actions.hasArrayJoin() || group_by_actions.hasStatefulFunctions() || group_by_actions.hasNonDeterministic()) return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - auto key_nodes = group_by_actions->findInOutpus(aggregating.getParams().keys); + auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ed4b1906635..a12fce95b10 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -943,7 +943,7 @@ SplitPartsWithRangesByPrimaryKeyResult splitPartsWithRangesByPrimaryKey( auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes()); auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false); - reorderColumns(*actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName()); + reorderColumns(actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName()); ExpressionActionsPtr expression_actions = std::make_shared(std::move(actions)); auto description = fmt::format( "filter values in ({}, {}]", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf"); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index eca3cc54ce9..bc878e7ee49 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -799,7 +799,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); + auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -848,16 +848,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); } -static ActionsDAGPtr createProjection(const Block & header) +static ActionsDAG createProjection(const Block & header) { - return std::make_unique(header.getNamesAndTypesList()); + return ActionsDAG(header.getNamesAndTypesList()); } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names, - ActionsDAGPtr & out_projection, + std::optional & out_projection, const InputOrderInfoPtr & input_order_info) { const auto & settings = context->getSettingsRef(); @@ -1171,7 +1171,7 @@ bool ReadFromMergeTree::doNotMergePartsAcrossPartitionsFinal() const } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection) + RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional & out_projection) { const auto & settings = context->getSettingsRef(); const auto & data_settings = data.getSettings(); @@ -1212,7 +1212,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); + auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); if (prewhere_info) { @@ -1333,7 +1333,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (!merging_pipes.empty() && !no_merging_pipes.empty()) { - out_projection = nullptr; /// We do projection here + out_projection = {}; /// We do projection here Pipes pipes; pipes.resize(2); pipes[0] = Pipe::unitePipes(std::move(merging_pipes)); @@ -1519,7 +1519,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. - query_info.filter_actions_dag = std::move(filter_actions_dag); + if (filter_actions_dag) + query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); buildIndexes( indexes, @@ -1833,7 +1834,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const } Pipe ReadFromMergeTree::spreadMarkRanges( - RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) + RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional & result_projection) { const bool final = isQueryWithFinal(); Names column_names_to_read = result.column_names_to_read; @@ -1894,7 +1895,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( } } -Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection) +Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, std::optional & result_projection) { auto && parts_with_ranges = std::move(result.parts_with_ranges); @@ -1983,7 +1984,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. /// NOTE: It may lead to double computation of expressions. - ActionsDAGPtr result_projection; + std::optional result_projection; Pipe pipe = output_each_partition_through_separate_port ? groupStreamsByPartition(result, result_projection) @@ -2000,7 +2001,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(ActionsDAG::clone(result.sampling.filter_expression.get())); + auto sampling_actions = std::make_shared(std::move(*ActionsDAG::clone(result.sampling.filter_expression.get()))); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2013,12 +2014,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons Block cur_header = pipe.getHeader(); - auto append_actions = [&result_projection](ActionsDAGPtr actions) + auto append_actions = [&result_projection](ActionsDAG actions) { if (!result_projection) result_projection = std::move(actions); else - result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); + result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(actions)); }; if (result_projection) @@ -2038,7 +2039,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(ActionsDAG::clone(result_projection)); + auto projection_actions = std::make_shared(std::move(*result_projection)); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2133,7 +2134,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); expression->describeActions(format_settings.out, prefix); } @@ -2142,7 +2143,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); expression->describeActions(format_settings.out, prefix); } } @@ -2168,7 +2169,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2178,7 +2179,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index e32507e1f22..a12f53924c3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -243,9 +243,9 @@ private: Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings); Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit); - Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection); + Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional & result_projection); - Pipe groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection); + Pipe groupStreamsByPartition(AnalysisResult & result, std::optional & result_projection); Pipe spreadMarkRangesAmongStreams(RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names); @@ -253,13 +253,13 @@ private: RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names, - ActionsDAGPtr & out_projection, + std::optional & out_projection, const InputOrderInfoPtr & input_order_info); bool doNotMergePartsAcrossPartitionsFinal() const; Pipe spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); + RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional & out_projection); ReadFromMergeTree::AnalysisResult getAnalysisResult() const; diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 90fe609a17d..ca98f7c2110 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -441,7 +441,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() chassert(numbers_storage.step != UInt64{0}); /// Build rpn of query filters - KeyCondition condition(filter_actions_dag.get(), context, column_names, key_expression); + KeyCondition condition(filter_actions_dag ? &*filter_actions_dag : nullptr, context, column_names, key_expression); if (condition.extractPlainRanges(ranges)) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index 79b225e7f93..55c9b5e442e 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 91b62efa860..f7a030c0628 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -33,8 +33,8 @@ public: { } - const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; } - ActionsDAGPtr detachFilterActionsDAG() { return std::move(filter_actions_dag); } + const std::optional & getFilterActionsDAG() const { return filter_actions_dag; } + std::optional detachFilterActionsDAG() { return std::move(filter_actions_dag); } const SelectQueryInfo & getQueryInfo() const { return query_info; } const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } @@ -81,7 +81,7 @@ protected: ContextPtr context; std::optional limit; - ActionsDAGPtr filter_actions_dag; + std::optional filter_actions_dag; private: /// Will be cleared after applyFilters() is called. diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 19632b1862f..4aa4f10ac86 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -28,7 +28,7 @@ TotalsHavingStep::TotalsHavingStep( const DataStream & input_stream_, const AggregateDescriptions & aggregates_, bool overflow_row_, - const ActionsDAGPtr & actions_dag_, + std::optional actions_dag_, const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, @@ -38,7 +38,7 @@ TotalsHavingStep::TotalsHavingStep( input_stream_, TotalsHavingTransform::transformHeader( input_stream_.header, - actions_dag_.get(), + actions_dag_ ? &*actions_dag_ : nullptr, filter_column_, remove_filter_, final_, @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(ActionsDAG::clone(actions_dag_)) + , actions_dag(std::move(actions_dag_)) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(std::move(*actions_dag), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); map.add("Expression", expression->toTree()); } } @@ -128,7 +128,7 @@ void TotalsHavingStep::updateOutputStream() input_streams.front(), TotalsHavingTransform::transformHeader( input_streams.front().header, - actions_dag.get(), + getActions(), filter_column_name, remove_filter, final, diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 52ef5437701..927b8d99de3 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -18,7 +19,7 @@ public: const DataStream & input_stream_, const AggregateDescriptions & aggregates_, bool overflow_row_, - const ActionsDAGPtr & actions_dag_, + std::optional actions_dag_, const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, @@ -32,7 +33,7 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getActions() const { return actions_dag; } + const ActionsDAG * getActions() const { return actions_dag ? &*actions_dag : nullptr; } private: void updateOutputStream() override; @@ -40,7 +41,7 @@ private: const AggregateDescriptions aggregates; bool overflow_row; - ActionsDAGPtr actions_dag; + std::optional actions_dag; String filter_column_name; bool remove_filter; TotalsMode totals_mode; diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index fcf576637ff..cfd3eb236b7 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -16,13 +16,13 @@ protected: /// Represents pushed down filters in source std::shared_ptr key_condition; - void setKeyConditionImpl(const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & keys) + void setKeyConditionImpl(const std::optional & filter_actions_dag, ContextPtr context, const Block & keys) { key_condition = std::make_shared( - filter_actions_dag, + filter_actions_dag ? &*filter_actions_dag : nullptr, context, keys.getNames(), - std::make_shared(std::make_unique(keys.getColumnsWithTypeAndName()))); + std::make_shared(ActionsDAG(keys.getColumnsWithTypeAndName()))); } public: @@ -33,6 +33,6 @@ public: virtual void setKeyCondition(const std::shared_ptr & key_condition_) { key_condition = key_condition_; } /// Set key_condition created by filter_actions_dag and context. - virtual void setKeyCondition(const ActionsDAGPtr & /*filter_actions_dag*/, ContextPtr /*context*/) { } + virtual void setKeyCondition(const std::optional & /*filter_actions_dag*/, ContextPtr /*context*/) { } }; } diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp index 7945b3999c1..da4d3a0041b 100644 --- a/src/Processors/Transforms/AddingDefaultsTransform.cpp +++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp @@ -178,7 +178,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk) auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false); if (dag) { - auto actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); + auto actions = std::make_shared(std::move(*dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); actions->execute(evaluate_block); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index bbe57fc6441..36ffc515f43 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(ActionsDAG::clone(interpolate_description->actions)); + interpolate_actions = std::make_shared(std::move(*ActionsDAG::clone(&interpolate_description->actions))); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 2cd51259549..da5a45f36d5 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -516,7 +516,7 @@ void StorageHive::initMinMaxIndexExpression() partition_names = partition_name_types.getNames(); partition_types = partition_name_types.getTypes(); partition_minmax_idx_expr = std::make_shared( - std::make_unique(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); + ActionsDAG(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); } NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); @@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression() hivefile_name_types.push_back(column); } hivefile_minmax_idx_expr = std::make_shared( - std::make_unique(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); + ActionsDAG(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); } ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) @@ -583,7 +583,7 @@ static HiveFilePtr createHiveFile( HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -647,7 +647,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(filter_actions_dag.get(), getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -681,7 +681,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv HiveFilePtr StorageHive::getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level) const @@ -715,7 +715,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(filter_actions_dag.get(), getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply @@ -828,7 +828,7 @@ void ReadFromHive::createFiles() if (hive_files) return; - hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context); + hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag ? &*filter_actions_dag : nullptr, hive_table_metadata, fs, context); LOG_INFO(log, "Collect {} hive files to read", hive_files->size()); } @@ -950,7 +950,7 @@ void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const Bui HiveFiles StorageHive::collectHiveFiles( size_t max_threads, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -1023,12 +1023,12 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad std::optional StorageHive::totalRows(const Settings & settings) const { /// query_info is not used when prune_level == PruneLevel::None - return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None); + return totalRowsImpl(settings, {}, getContext(), PruneLevel::None); } -std::optional StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const +std::optional StorageHive::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const { - return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition); + return totalRowsImpl(context_->getSettingsRef(), &filter_actions_dag, context_, PruneLevel::Partition); } void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const @@ -1043,7 +1043,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt } std::optional -StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const +StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const { /// Row-based format like Text doesn't support totalRowsByPartitionPredicate if (!supportsSubsetOfColumns()) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 8a457dd6e01..e16df22e138 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -57,7 +57,7 @@ public: bool supportsSubsetOfColumns() const; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; protected: @@ -90,7 +90,7 @@ private: HiveFiles collectHiveFiles( size_t max_threads, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -98,7 +98,7 @@ private: HiveFiles collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -107,7 +107,7 @@ private: HiveFilePtr getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; @@ -115,7 +115,7 @@ private: void lazyInitialize(); std::optional - totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; + totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; String hive_metastore_url; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 1f7ac23eb82..57f79a2cd7f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -236,7 +236,7 @@ StorageID IStorage::getStorageID() const return storage_id; } -ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const +ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const { return {}; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 98afd844046..c86f18d5d3b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -135,7 +135,7 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } - virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const; + virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const; /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// This is needed for engines whose aggregates data from multiple tables, like Merge. @@ -682,7 +682,7 @@ public: virtual std::optional totalRows(const Settings &) const { return {}; } /// Same as above but also take partition predicate into account. - virtual std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; } + virtual std::optional totalRowsByPartitionPredicate(const ActionsDAG &, ContextPtr) const { return {}; } /// If it is possible to quickly determine exact number of bytes for the table on storage: /// - memory (approximated, resident) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 94319aef3b8..88783246e10 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -231,7 +231,7 @@ bool traverseDAGFilter( } std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context) + const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context) { if (!filter_actions_dag) return {{}, true}; diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index e20a1ce4f37..64108290270 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -22,7 +22,7 @@ std::pair getFilterKeys( const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context); + const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context); template void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index e03ecc05064..7e43966556e 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -160,7 +160,7 @@ KeyDescription KeyDescription::buildEmptyKey() { KeyDescription result; result.expression_list_ast = std::make_shared(); - result.expression = std::make_shared(std::make_unique(), ExpressionActionsSettings{}); + result.expression = std::make_shared(ActionsDAG(), ExpressionActionsSettings{}); return result; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4ad7f6ef991..264b2b397f4 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -163,8 +163,8 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns if (dag) { dag->addMaterializingOutputActions(); - auto actions = std::make_shared< - ExpressionActions>(std::move(dag), + auto actions = std::make_shared( + std::move(*dag), ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef())); actions->execute(additional_columns); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 1efded3b064..d781345d834 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -628,7 +628,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { const auto & index_hint_dag = index_hint->getActions(); - children = index_hint_dag->getOutputs(); + children = index_hint_dag.getOutputs(); for (auto & arg : children) arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); @@ -729,7 +729,7 @@ Block KeyCondition::getBlockWithConstants( if (syntax_analyzer_result) { auto actions = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActionsDAG(); - for (const auto & action_node : actions->getOutputs()) + for (const auto & action_node : actions.getOutputs()) { if (action_node->column) result.insert(ColumnWithTypeAndName{action_node->column, action_node->result_type, action_node->result_name}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7b642c34f37..334c8c9c5ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -472,7 +472,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const } ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByPredicate( - const StorageSnapshotPtr & storage_snapshot, const ActionsDAGPtr & filter_dag, ContextPtr local_context) const + const StorageSnapshotPtr & storage_snapshot, const ActionsDAG * filter_dag, ContextPtr local_context) const { if (!local_context->getSettings().allow_statistics_optimize) return {}; @@ -487,7 +487,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP ASTPtr expression_ast; ConditionSelectivityEstimator result; - PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag.get(), local_context); + PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context); if (partition_pruner.isUseless()) { @@ -746,7 +746,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti if (!partition_key.column_names.empty()) partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); - return std::make_shared(std::make_unique(partition_key_columns), settings); + return std::make_shared(ActionsDAG(partition_key_columns), settings); } Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) @@ -1134,7 +1134,7 @@ Block MergeTreeData::getBlockWithVirtualsForFilter( std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( - const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const + const ActionsDAG & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const { if (parts.empty()) return 0; @@ -1142,7 +1142,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr); if (!filter_dag) return {}; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 52916d85fef..e490e4b0bf9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -426,7 +426,7 @@ public: bool supportsPrewhere() const override { return true; } - ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const override; + ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const override; bool supportsFinal() const override; @@ -1227,7 +1227,7 @@ protected: boost::iterator_range range, const ColumnsDescription & storage_columns); std::optional totalRowsByPartitionPredicateImpl( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; + const ActionsDAG & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; static decltype(auto) getStateModifier(DataPartState state) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 61b8b6fdaa8..5a5b6d4a6e1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -428,7 +428,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( ASTPtr query = sampling.filter_function; auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); - sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); + sampling.filter_expression = std::make_shared(ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false)); } } @@ -466,7 +466,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( dag.get(), context, sample.getNames(), - std::make_shared(std::make_unique(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), + std::make_shared(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), {}}); } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 8d4ef69b1b9..ca31ffc9de5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,15 +265,15 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = ActionsDAG::clone(filter_dag); - const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); + auto filter_actions_dag = std::move(*ActionsDAG::clone(filter_dag)); + const auto * filter_actions_dag_node = filter_actions_dag.getOutputs().at(0); std::unordered_map node_to_result_node; - filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); + filter_actions_dag.getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); - filter_actions_dag->removeUnusedActions(); + filter_actions_dag.removeUnusedActions(); - actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; + actions_output_column_name = filter_actions_dag.getOutputs().at(0)->result_name; actions = std::make_shared(std::move(filter_actions_dag)); } @@ -306,7 +306,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx } -static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, const ActionsDAGPtr & result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage) +static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, ActionsDAG * result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage) { chassert(node.type == ActionsDAG::ActionType::FUNCTION); if (node.function_base->getName() != "indexHint") @@ -316,17 +316,17 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node const auto & adaptor = typeid_cast(*node.function_base); const auto & index_hint = typeid_cast(*adaptor.getFunction()); if (!result_dag_or_null) - return index_hint.getActions()->getOutputs(); + return index_hint.getActions().getOutputs(); /// Import the DAG and map argument pointers. - ActionsDAGPtr actions_clone = ActionsDAG::clone(index_hint.getActions()); + auto actions_clone = std::move(*ActionsDAG::clone(&index_hint.getActions())); chassert(storage); - result_dag_or_null->mergeNodes(std::move(*actions_clone), storage); + result_dag_or_null->mergeNodes(std::move(actions_clone), storage); return *storage; } const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const { @@ -348,7 +348,7 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA atom_node_ptr->type == ActionsDAG::ActionType::FUNCTION) { auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); - result_node = &result_dag->addFunction(bit_wrapper_function, {atom_node_ptr}, {}); + result_node = &result_dag.addFunction(bit_wrapper_function, {atom_node_ptr}, {}); } } else @@ -359,14 +359,14 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA unknown_field_column_with_type.type = std::make_shared(); unknown_field_column_with_type.column = unknown_field_column_with_type.type->createColumnConst(1, UNKNOWN_FIELD); - result_node = &result_dag->addColumn(unknown_field_column_with_type); + result_node = &result_dag.addColumn(unknown_field_column_with_type); } node_to_result_node.emplace(&node, result_node); return *result_node; } -const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAGPtr & result_dag, const ContextPtr & context) const +const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAG & result_dag, const ContextPtr & context) const { /// Function, literal or column @@ -386,7 +386,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA const auto * result_node = node_to_check; if (node.type != ActionsDAG::ActionType::INPUT) - result_node = &result_dag->addInput(column_name, node.result_type); + result_node = &result_dag.addInput(column_name, node.result_type); return result_node; } @@ -407,11 +407,11 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA return nullptr; } - return &result_dag->addFunction(node.function_base, children, {}); + return &result_dag.addFunction(node.function_base, children, {}); } const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const { @@ -429,7 +429,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio auto function_name = node_to_check->function->getName(); ActionsDAG::NodeRawConstPtrs temp_ptrs_to_argument; - const auto & arguments = getArguments(*node_to_check, result_dag, &temp_ptrs_to_argument); + const auto & arguments = getArguments(*node_to_check, &result_dag, &temp_ptrs_to_argument); size_t arguments_size = arguments.size(); if (function_name == "not") @@ -440,7 +440,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node); auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); - return &result_dag->addFunction(bit_swap_last_two_function, {argument}, {}); + return &result_dag.addFunction(bit_swap_last_two_function, {argument}, {}); } else if (function_name == "and" || function_name == "indexHint" || function_name == "or") { @@ -468,7 +468,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio const auto * before_last_argument = children.back(); children.pop_back(); - last_argument = &result_dag->addFunction(function, {before_last_argument, last_argument}, {}); + last_argument = &result_dag.addFunction(function, {before_last_argument, last_argument}, {}); } return last_argument; diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index abd40b3cf9d..03b02515e47 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -93,16 +93,16 @@ public: ~MergeTreeIndexConditionSet() override = default; private: const ActionsDAG::Node & traverseDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const; const ActionsDAG::Node * atomFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context) const; const ActionsDAG::Node * operatorFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index e924f853524..aec2f988e8d 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter), actions_settings), + .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions), actions_settings), + .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 98b35a3ca2c..15917d59c9f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -349,7 +349,7 @@ public: MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool apply_deleted_mask_, - ActionsDAGPtr filter_, + std::optional filter_, ContextPtr context_, LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) @@ -376,7 +376,7 @@ public: { const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - KeyCondition key_condition(filter.get(), context, primary_key_column_names, primary_key.expression); + KeyCondition key_condition(&*filter, context, primary_key_column_names, primary_key.expression); LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) @@ -417,7 +417,7 @@ private: MergeTreeData::DataPartPtr data_part; Names columns_to_read; bool apply_deleted_mask; - ActionsDAGPtr filter; + std::optional filter; ContextPtr context; LoggerPtr log; }; @@ -430,7 +430,7 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, Names columns_to_read, bool apply_deleted_mask, - ActionsDAGPtr filter, + std::optional filter, ContextPtr context, LoggerPtr log) { diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index e6f055f776c..1b05512b9a3 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -38,7 +38,7 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, Names columns_to_read, bool apply_deleted_mask, - ActionsDAGPtr filter, + std::optional filter, ContextPtr context, LoggerPtr log); diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 25596b42951..116edf5b9cb 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -349,7 +349,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction PrewhereExprStep new_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(step.actions), actions_settings), + .actions = std::make_shared(std::move(*step.actions), actions_settings), .filter_column_name = step.column_name, /// Don't remove if it's in the list of original outputs .remove_filter_column = diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index a9a5fddace4..8c389f00780 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -112,7 +112,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); } -MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, +MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAG & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final) @@ -126,7 +126,7 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize; RPNBuilderTreeContext tree_context(context); - RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); + RPNBuilderTreeNode node(&filter_dag.findInOutputs(filter_column_name), tree_context); auto optimize_result = optimizeImpl(node, where_optimizer_context); if (!optimize_result) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index ba6b4660924..a3d035675c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -52,7 +52,7 @@ public: bool fully_moved_to_prewhere = false; }; - FilterActionsOptimizeResult optimize(const ActionsDAGPtr & filter_dag, + FilterActionsOptimizeResult optimize(const ActionsDAG & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 4a18d606bb7..915a0e84902 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -398,7 +398,7 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const { const auto * adaptor = typeid_cast(dag_node->function_base.get()); const auto * index_hint = typeid_cast(adaptor->getFunction().get()); - return index_hint->getActions()->getOutputs().size(); + return index_hint->getActions().getOutputs().size(); } return dag_node->children.size(); @@ -426,7 +426,7 @@ RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const { const auto & adaptor = typeid_cast(*dag_node->function_base); const auto & index_hint = typeid_cast(*adaptor.getFunction()); - return RPNBuilderTreeNode(index_hint.getActions()->getOutputs()[index], tree_context); + return RPNBuilderTreeNode(index_hint.getActions().getOutputs()[index], tree_context); } return RPNBuilderTreeNode(dag_node->children[index], tree_context); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 503f542f2bd..d114608d8f1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -77,9 +77,9 @@ StorageObjectStorageSource::~StorageObjectStorageSource() create_reader_pool->wait(); } -void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +void StorageObjectStorageSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag.get(), context_, read_from_format_info.format_header); + setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); } std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index fd7c7aa7102..01ce980feaa 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -45,7 +45,7 @@ public: String getName() const override { return name; } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; Chunk generate() override; diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index e1d52eefc20..393d3f3fbb9 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -79,7 +79,7 @@ void readFinalFromNestedStorage( auto step = std::make_unique( query_plan.getCurrentDataStream(), - actions, + std::move(actions), filter_column_name, false); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 5f48d5e795e..5276870c037 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -46,9 +46,9 @@ struct PrewhereInfo { /// Actions for row level security filter. Applied separately before prewhere_actions. /// This actions are separate because prewhere condition should not be executed over filtered rows. - ActionsDAGPtr row_level_filter; + std::optional row_level_filter; /// Actions which are executed on block in order to get filter column for prewhere step. - ActionsDAGPtr prewhere_actions; + std::optional prewhere_actions; String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; @@ -56,7 +56,7 @@ struct PrewhereInfo bool generated_by_optimizer = false; PrewhereInfo() = default; - explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(std::optional prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} std::string dump() const; @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = ActionsDAG::clone(row_level_filter); + prewhere_info->row_level_filter = std::move(*ActionsDAG::clone(&*row_level_filter)); if (prewhere_actions) - prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); + prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(&*prewhere_actions)); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; @@ -93,7 +93,7 @@ struct FilterInfo /// Same as FilterInfo, but with ActionsDAG. struct FilterDAGInfo { - ActionsDAGPtr actions; + std::optional actions; String column_name; bool do_remove_column = false; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 695b31d0c80..fdddd84ab59 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -312,7 +312,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*ActionsDAG::clone(actions_dag)), + std::move(*ActionsDAG::clone(&actions_dag)), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -321,7 +321,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*ActionsDAG::clone(actions_dag)), + std::move(*ActionsDAG::clone(&actions_dag)), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -353,7 +353,7 @@ void StorageBuffer::read( header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag)); converting->setStepDescription("Convert destination table columns to Buffer table structure"); query_plan.addStep(std::move(converting)); @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(ActionsDAG::clone(query_info.prewhere_info->row_level_filter), actions_settings), + std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->row_level_filter)), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(ActionsDAG::clone(query_info.prewhere_info->prewhere_actions), actions_settings), + std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->prewhere_actions)), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -472,7 +472,7 @@ void StorageBuffer::read( result_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); query_plan.addStep(std::move(converting)); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3d91da240cc..6f8a9189941 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1074,7 +1074,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu return pipeline; } -static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) +static std::optional getFilterFromQuery(const ASTPtr & ast, ContextPtr context) { QueryPlan plan; SelectQueryOptions options; @@ -1118,7 +1118,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) } if (!source) - return nullptr; + return {}; return source->detachFilterActionsDAG(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2422bcd700b..4611371a471 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1233,9 +1233,9 @@ StorageFileSource::~StorageFileSource() beforeDestroy(); } -void StorageFileSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +void StorageFileSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); + setKeyConditionImpl(filter_actions_dag, context_, block_for_format); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ac094aeb489..e9424527997 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -265,7 +265,7 @@ private: return storage->getName(); } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; bool tryGetCountFromCache(const struct stat & file_stat); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 316f398b476..8b6a9a4d4bb 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -273,8 +273,8 @@ void StorageMaterializedView::read( * They may be added in case of distributed query with JOIN. * In that case underlying table returns joined columns as well. */ - converting_actions->removeUnusedActions(); - auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); + converting_actions.removeUnusedActions(); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(converting_actions)); converting_step->setStepDescription("Convert target table structure to MaterializedView structure"); query_plan.addStep(std::move(converting_step)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c3fdad3a8f2..374abd0b0a5 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -407,7 +407,7 @@ void ReadFromMerge::addFilter(FilterDAGInfo filter) { output_stream->header = FilterTransform::transformHeader( output_stream->header, - filter.actions.get(), + filter.actions ? &*filter.actions : nullptr, filter.column_name, filter.do_remove_column); pushed_down_filters.push_back(std::move(filter)); @@ -628,7 +628,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); - column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); + column_names_as_aliases = alias_actions.getRequiredColumns().getNames(); if (column_names_as_aliases.empty()) column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - ActionsDAG::clone(filter_info.actions), + std::move(*ActionsDAG::clone(&*filter_info.actions)), filter_info.column_name, filter_info.do_remove_column); @@ -1060,7 +1060,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1074,7 +1074,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1089,7 +1089,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1102,7 +1102,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1240,7 +1240,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(ActionsDAG::clone(actions_dag), + filter_actions = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag)), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1278,12 +1278,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(ActionsDAG::clone(actions_dag), filter_column_name); + step->addFilter(ActionsDAG::clone(&actions_dag), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), ActionsDAG::clone(actions_dag), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&actions_dag)), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } @@ -1476,7 +1476,7 @@ void ReadFromMerge::convertAndFilterSourceStream( { pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - auto actions_dag = std::make_unique(pipe_columns); + ActionsDAG actions_dag(pipe_columns); QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context); query_tree->setAlias(alias.name); @@ -1485,12 +1485,12 @@ void ReadFromMerge::convertAndFilterSourceStream( query_analysis_pass.run(query_tree, local_context); PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); - const auto & nodes = actions_visitor.visit(*actions_dag, query_tree); + const auto & nodes = actions_visitor.visit(actions_dag, query_tree); if (nodes.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); - actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); + actions_dag.addOrReplaceInOutputs(actions_dag.addAlias(*nodes.front(), alias.name)); auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } @@ -1506,7 +1506,7 @@ void ReadFromMerge::convertAndFilterSourceStream( auto dag = std::make_shared(pipe_columns); auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } } @@ -1524,7 +1524,7 @@ void ReadFromMerge::convertAndFilterSourceStream( header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), convert_actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(convert_actions_dag)); child.plan.addStep(std::move(expression_step)); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 94b34256d02..d6f2deca7fd 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -225,7 +225,7 @@ private: private: std::string filter_column_name; // complex filter, may contain logic operations - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; ExpressionActionsPtr filter_actions; StorageMetadataPtr storage_metadata_snapshot; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..b1a8a81914c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -245,7 +245,7 @@ std::optional StorageMergeTree::totalRows(const Settings &) const return getTotalActiveSizeInRows(); } -std::optional StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const +std::optional StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr local_context) const { auto parts = getVisibleDataPartsVector(local_context); return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4d819508934..56324449b34 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -65,7 +65,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings &) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr) const override; std::optional totalBytes(const Settings &) const override; std::optional totalBytesUncompressed(const Settings &) const override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..b472710b6d8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5556,7 +5556,7 @@ std::optional StorageReplicatedMergeTree::totalRows(const Settings & set return res; } -std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const +std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr local_context) const { DataPartsVector parts; foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f96206ce657..2e54f17d5d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -159,7 +159,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context) const override; std::optional totalBytes(const Settings & settings) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 9507eb6ed8a..345dd62c687 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -112,7 +112,7 @@ public: auto step = std::make_unique( query_plan.getCurrentDataStream(), - convert_actions_dag); + std::move(convert_actions_dag)); step->setStepDescription("Converting columns"); query_plan.addStep(std::move(step)); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index c336f597f41..ec1f803750e 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -185,9 +185,9 @@ public: String getName() const override { return name; } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override { - setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); + setKeyConditionImpl(filter_actions_dag, context_, block_for_format); } Chunk generate() override; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 4d73f8e5c87..c1ca6244866 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -48,13 +48,13 @@ Pipe StorageValues::read( if (!prepared_pipe.empty()) { - auto dag = std::make_unique(prepared_pipe.getHeader().getColumnsWithTypeAndName()); + ActionsDAG dag(prepared_pipe.getHeader().getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(column_names.size()); for (const auto & name : column_names) - outputs.push_back(dag->getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); + outputs.push_back(dag.getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); - dag->getOutputs().swap(outputs); + dag.getOutputs().swap(outputs); auto expression = std::make_shared(std::move(dag)); prepared_pipe.addSimpleTransform([&](const Block & header) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 2c0d5c5ca85..e2c4d67c8d1 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -177,8 +177,8 @@ void StorageView::read( /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - auto materializing_actions = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - materializing_actions->addMaterializingOutputActions(); + ActionsDAG materializing_actions(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + materializing_actions.addMaterializingOutputActions(); auto materializing = std::make_unique(query_plan.getCurrentDataStream(), std::move(materializing_actions)); materializing->setStepDescription("Materialize constants after VIEW subquery"); @@ -203,7 +203,7 @@ void StorageView::read( expected_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); converting->setStepDescription("Convert VIEW subquery result to VIEW table structure"); query_plan.addStep(std::move(converting)); } diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index ba7433fb9ae..f4e6fe3df5f 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -276,7 +276,7 @@ public: StackTraceSource( const Names & column_names, Block header_, - ActionsDAGPtr && filter_dag_, + std::optional filter_dag_, ContextPtr context_, UInt64 max_block_size_, LoggerPtr log_) @@ -422,7 +422,7 @@ protected: private: ContextPtr context; Block header; - const ActionsDAGPtr filter_dag; + const std::optional filter_dag; const ActionsDAG::Node * predicate; const size_t max_block_size; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 56f65b57367..a32eef20aed 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -176,12 +176,12 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); - const auto * col = &dag->findInOutputs(ast->getColumnName()); + const auto * col = &dag.findInOutputs(ast->getColumnName()); if (col->result_name != ttl_string) - col = &dag->addAlias(*col, ttl_string); + col = &dag.addAlias(*col, ttl_string); - dag->getOutputs() = {col}; - dag->removeUnusedActions(); + dag.getOutputs() = {col}; + dag.removeUnusedActions(); result.expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1bd5e80a4f9..7f54c6a6ee3 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,7 +80,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(ActionsDAG::clone(dag)); + auto actions = std::make_shared(std::move(*ActionsDAG::clone(dag))); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); @@ -318,9 +318,9 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = ActionsDAG::clone(index_hint->getActions()); + auto index_hint_dag = std::move(*ActionsDAG::clone(&index_hint->getActions())); ActionsDAG::NodeRawConstPtrs atoms; - for (const auto & output : index_hint_dag->getOutputs()) + for (const auto & output : index_hint_dag.getOutputs()) if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) atoms.push_back(child_copy); @@ -331,13 +331,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( if (atoms.size() > 1) { FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - res = &index_hint_dag->addFunction(func_builder_and, atoms, {}); + res = &index_hint_dag.addFunction(func_builder_and, atoms, {}); } if (!res->result_type->equals(*node->result_type)) - res = &index_hint_dag->addCast(*res, node->result_type, {}); + res = &index_hint_dag.addCast(*res, node->result_type, {}); - additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(*index_hint_dag))); + additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(index_hint_dag))); return res; } } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8f39f0da5af..30ae1f95593 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1130,7 +1130,7 @@ void StorageWindowView::read( { auto converting_actions = ActionsDAG::makeConvertingActions( target_header.getColumnsWithTypeAndName(), wv_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(converting_actions)); converting_step->setStepDescription("Convert Target table structure to WindowView structure"); query_plan.addStep(std::move(converting_step)); } From 1237f93182db21f00df9ca7913619ee63d75850b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jul 2024 15:06:52 +0000 Subject: [PATCH 017/321] Fixing some crashes. --- src/Interpreters/ExpressionActions.cpp | 2 +- src/Planner/Planner.cpp | 8 +++++--- src/Planner/PlannerJoins.cpp | 8 ++++---- src/Planner/PlannerJoins.h | 4 ++-- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 399f4f2ff4f..1c6c3f2556b 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -59,7 +59,7 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) - actions_dag->compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); + actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); #endif linearizeActions(lazy_executed_nodes); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 48e42099ce8..0b10cef82ce 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -212,9 +212,11 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & if (!read_from_dummy) continue; - auto filter_actions = read_from_dummy->detachFilterActionsDAG(); - const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); - res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; + if (auto filter_actions = read_from_dummy->detachFilterActionsDAG()) + { + const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); + res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; + } } return res; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index db9678d91a6..7772336f7c0 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -588,11 +588,11 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = std::move(left_join_actions); - //result.left_join_tmp_expression_actions = std::move(left_join_actions); + result.left_join_expressions_actions = std::move(*ActionsDAG::clone(&left_join_actions)); + result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = std::move(right_join_actions); - //result.right_join_tmp_expression_actions = std::move(right_join_actions); + result.right_join_expressions_actions = std::move(*ActionsDAG::clone(&right_join_actions)); + result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); if (is_inequal_join) diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 3735c373acc..d8665ab7739 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -182,8 +182,8 @@ struct JoinClausesAndActions /// Join clauses. Actions dag nodes point into join_expression_actions. JoinClauses join_clauses; /// Whole JOIN ON section expressions - // ActionsDAGPtr left_join_tmp_expression_actions; - // ActionsDAGPtr right_join_tmp_expression_actions; + ActionsDAG left_join_tmp_expression_actions; + ActionsDAG right_join_tmp_expression_actions; /// Left join expressions actions ActionsDAG left_join_expressions_actions; /// Right join expressions actions diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bc878e7ee49..9ca79fde26f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1520,7 +1520,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); + query_info.filter_actions_dag = std::make_shared(std::move(*ActionsDAG::clone(&*filter_actions_dag))); buildIndexes( indexes, From 11a8de50a6283277c585fa2bad74aad1712fb1f2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 Jul 2024 08:56:35 +0000 Subject: [PATCH 018/321] Revert "Disable broken cases from 02911_join_on_nullsafe_optimization" This reverts commit 513ce9fa2f3bb0d2cc1774a07272a249b40f475f. --- ...2911_join_on_nullsafe_optimization.reference | 17 +++++++++++++---- .../02911_join_on_nullsafe_optimization.sql | 5 ++--- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 4eb7e74446d..f0463509b80 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -36,10 +36,19 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N 3 3 3 33 \N \N \N \N -- aliases defined in the join condition are valid --- FIXME(@vdimir) broken query formatting for the following queries: --- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; --- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; - +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +1 42 \N \N \N 0 +2 2 2 2 1 1 +3 3 3 33 1 1 +\N \N 4 42 \N 0 +\N \N \N \N \N 1 +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +1 42 \N \N \N 0 +2 2 2 2 1 1 +3 3 3 33 1 1 +\N \N 4 42 \N 0 +\N \N \N \N \N 0 +\N \N \N \N \N 0 -- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; 2 2 2 2 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index f7813e2a1b4..67918f4302f 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -36,9 +36,8 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- aliases defined in the join condition are valid --- FIXME(@vdimir) broken query formatting for the following queries: --- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; --- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; From 6e762d404456debca4ee2d5ccce94deb32c3fbad Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 Jul 2024 08:57:33 +0000 Subject: [PATCH 019/321] Fix aliased JOIN ON expression formatting --- src/Parsers/ASTTablesInSelectQuery.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index d22a4eca0fc..dbb2a008bae 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -235,7 +235,12 @@ void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatS else if (on_expression) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); + bool on_has_alias = !on_expression->tryGetAlias().empty(); + if (on_has_alias) + settings.ostr << "("; on_expression->formatImpl(settings, state, frame); + if (on_has_alias) + settings.ostr << ")"; } } From a6e737ef2afc7fb18d661295e6f84cc3e0478ae1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 17:19:30 +0000 Subject: [PATCH 020/321] Cleaner FilterDAGInfo. --- src/Interpreters/ActionsDAG.cpp | 25 +++++++------- src/Interpreters/ActionsDAG.h | 2 +- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 18 +++++------ src/Planner/Planner.cpp | 4 +-- src/Planner/PlannerJoinTree.cpp | 36 +++++++++------------ src/Storages/IStorage.cpp | 6 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/StorageMerge.cpp | 6 ++-- 11 files changed, 49 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 04be9d23c32..4401c83549f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1249,31 +1249,30 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) { std::unordered_map old_to_new_nodes; - return ActionsDAG::clone(from, old_to_new_nodes); + if (from == nullptr) + return nullptr; + return std::make_unique(ActionsDAG::clone(*from, old_to_new_nodes)); } -ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes) +ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes) { - if (!from) - return nullptr; + ActionsDAG actions; - auto actions = std::make_unique(); - - for (const auto & node : from->nodes) + for (const auto & node : from.nodes) { - auto & copy_node = actions->nodes.emplace_back(node); + auto & copy_node = actions.nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; } - for (auto & node : actions->nodes) + for (auto & node : actions.nodes) for (auto & child : node.children) child = old_to_new_nodes[child]; - for (const auto & output_node : from->outputs) - actions->outputs.push_back(old_to_new_nodes[output_node]); + for (const auto & output_node : from.outputs) + actions.outputs.push_back(old_to_new_nodes[output_node]); - for (const auto & input_node : from->inputs) - actions->inputs.push_back(old_to_new_nodes[input_node]); + for (const auto & input_node : from.inputs) + actions.inputs.push_back(old_to_new_nodes[input_node]); return actions; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index cf6a91b9fe7..f428ca2f01c 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -263,7 +263,7 @@ public: static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } static ActionsDAGPtr clone(const ActionsDAG * from); - static ActionsDAGPtr clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes); + static ActionsDAG clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes); static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 1c6c3f2556b..dd1d2eb703e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -75,7 +75,7 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = std::move(*ActionsDAG::clone(&actions_dag, copy_map)); + copy->actions_dag = ActionsDAG::clone(actions_dag, copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 068b6f290fa..286eda14b3f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1922,7 +1922,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && additional_filter) { - Names columns_for_additional_filter = additional_filter->actions->getRequiredColumnsNames(); + Names columns_for_additional_filter = additional_filter->actions.getRequiredColumnsNames(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_additional_filter.begin(), columns_for_additional_filter.end()); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cde6e305005..e723e5f7982 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -178,12 +178,12 @@ FilterDAGInfoPtr generateFilterActions( filter_info->actions = std::move(analyzer.simpleSelectActions()->dag); filter_info->column_name = expr_list->children.at(0)->getColumnName(); - filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); + filter_info->actions.removeUnusedActions(NameSet{filter_info->column_name}); - for (const auto * node : filter_info->actions->getInputs()) - filter_info->actions->getOutputs().push_back(node); + for (const auto * node : filter_info->actions.getInputs()) + filter_info->actions.getOutputs().push_back(node); - auto required_columns_from_filter = filter_info->actions->getRequiredColumns(); + auto required_columns_from_filter = filter_info->actions.getRequiredColumns(); for (const auto & column : required_columns_from_filter) { @@ -1486,7 +1486,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), + std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1612,7 +1612,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), + std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1620,11 +1620,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*new_filter_info->actions)), + std::move(new_filter_info->actions), new_filter_info->column_name, new_filter_info->do_remove_column); @@ -2107,7 +2107,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(*analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 0b10cef82ce..ffed19185d3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1128,11 +1128,11 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto fake_table_expression = std::make_shared(std::move(storage), query_context); auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); - if (!filter_info.actions || !query_plan.isInitialized()) + if (!query_plan.isInitialized()) return; auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - std::move(*filter_info.actions), + std::move(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription("additional result filter"); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index fa3a3483a8e..3217d3461d3 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -458,7 +458,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end()); } -FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, +std::optional buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context, std::set & used_row_policies) @@ -479,7 +479,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); } -FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, +std::optional buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) { @@ -513,7 +513,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, } /// Apply filters from additional_table_filters setting -FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, +std::optional buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) @@ -789,9 +789,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres std::vector> where_filters; const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description) { - if (!filter_info.actions) - return; - bool is_final = table_expression_query_info.table_expression_modifiers && table_expression_query_info.table_expression_modifiers->hasFinal(); bool optimize_move_to_prewhere @@ -805,14 +802,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = std::move(*filter_info.actions); + prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = std::move(*filter_info.actions); + prewhere_info->row_level_filter = std::move(filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } @@ -830,17 +827,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); - if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&*row_policy_filter_info.actions)); - add_filter(row_policy_filter_info, "Row-level security filter"); + if (row_policy_filter_info) + { + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&row_policy_filter_info->actions)); + add_filter(*row_policy_filter_info, "Row-level security filter"); + } if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info - = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); - add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); + if (auto parallel_replicas_custom_key_filter_info= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context)) + add_filter(*parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else if (auto * distributed = typeid_cast(storage.get()); distributed && query_context->canUseParallelReplicasCustomKey(*distributed->getCluster())) @@ -850,9 +848,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getOriginalAlias(); - auto additional_filters_info - = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); - add_filter(additional_filters_info, "additional filter"); + if (auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context)) + add_filter(*additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage( query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); @@ -967,11 +964,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres for (auto && [filter_info, description] : where_filters) { if (query_plan.isInitialized() && - from_stage == QueryProcessingStage::FetchColumns && - filter_info.actions) + from_stage == QueryProcessingStage::FetchColumns) { auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - std::move(*filter_info.actions), + std::move(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription(description); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 57f79a2cd7f..4164608b4b5 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -340,10 +340,8 @@ std::string FilterDAGInfo::dump() const WriteBufferFromOwnString ss; ss << "FilterDAGInfo for column '" << column_name <<"', do_remove_column " << do_remove_column << "\n"; - if (actions) - { - ss << "actions " << actions->dumpDAG() << "\n"; - } + + ss << "actions " << actions.dumpDAG() << "\n"; return ss.str(); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 334c8c9c5ac..88fb52a94f2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7058,7 +7058,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS ActionDAGNodes filter_nodes; if (auto additional_filter_info = select.getAdditionalQueryInfo()) - filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); + filter_nodes.nodes.push_back(&additional_filter_info->actions.findInOutputs(additional_filter_info->column_name)); if (before_where) filter_nodes.nodes.push_back(&before_where->dag.findInOutputs(where_column_name)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 5276870c037..97b36115dfd 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -93,7 +93,7 @@ struct FilterInfo /// Same as FilterInfo, but with ActionsDAG. struct FilterDAGInfo { - std::optional actions; + ActionsDAG actions; String column_name; bool do_remove_column = false; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 374abd0b0a5..18e194491b8 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -407,7 +407,7 @@ void ReadFromMerge::addFilter(FilterDAGInfo filter) { output_stream->header = FilterTransform::transformHeader( output_stream->header, - filter.actions ? &*filter.actions : nullptr, + &filter.actions, filter.column_name, filter.do_remove_column); pushed_down_filters.push_back(std::move(filter)); @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*filter_info.actions)), + std::move(*ActionsDAG::clone(&filter_info.actions)), filter_info.column_name, filter_info.do_remove_column); @@ -1565,7 +1565,7 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { for (const auto & filter_info : pushed_down_filters) - added_filter_nodes.nodes.push_back(&filter_info.actions->findInOutputs(filter_info.column_name)); + added_filter_nodes.nodes.push_back(&filter_info.actions.findInOutputs(filter_info.column_name)); SourceStepWithFilter::applyFilters(added_filter_nodes); From fb7cf4ab93c991b3e2cd8a3e3e1c6cecf574b936 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 17:46:03 +0000 Subject: [PATCH 021/321] Better. --- src/Interpreters/ActionsDAG.cpp | 16 +++++++++++----- src/Interpreters/ActionsDAG.h | 4 +++- src/Interpreters/ExpressionActions.cpp | 2 +- src/Planner/CollectTableExpressionData.cpp | 10 +++++----- src/Planner/Planner.cpp | 4 ++-- src/Planner/PlannerJoinTree.cpp | 7 ++++--- src/Planner/TableExpressionData.h | 18 +++++++++--------- 7 files changed, 35 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4401c83549f..4f03a9e1602 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1251,14 +1251,20 @@ ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) std::unordered_map old_to_new_nodes; if (from == nullptr) return nullptr; - return std::make_unique(ActionsDAG::clone(*from, old_to_new_nodes)); + return std::make_unique(from->clone(old_to_new_nodes)); } -ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes) +ActionsDAG ActionsDAG::clone() const +{ + std::unordered_map old_to_new_nodes; + return clone(old_to_new_nodes); +} + +ActionsDAG ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const { ActionsDAG actions; - for (const auto & node : from.nodes) + for (const auto & node : nodes) { auto & copy_node = actions.nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; @@ -1268,10 +1274,10 @@ ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes); + + ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; + ActionsDAG clone() const; static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index dd1d2eb703e..113410b1480 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -75,7 +75,7 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = ActionsDAG::clone(actions_dag, copy_map); + copy->actions_dag = actions_dag.clone(copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 162d3fe8d11..1d85476636c 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -335,22 +335,22 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr collect_source_columns_visitor.setKeepAliasColumns(false); collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); - auto prewhere_actions_dag = std::make_unique(); + ActionsDAG prewhere_actions_dag; QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node); + auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "Invalid PREWHERE. Expected single boolean expression. In query {}", query_node->formatASTForErrorMessage()); - prewhere_actions_dag->getOutputs().push_back(expression_nodes.back()); + prewhere_actions_dag.getOutputs().push_back(expression_nodes.back()); - for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs()) + for (const auto & prewhere_input_node : prewhere_actions_dag.getInputs()) if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name)) - prewhere_actions_dag->getOutputs().push_back(prewhere_input_node); + prewhere_actions_dag.getOutputs().push_back(prewhere_input_node); table_expression_data.setPrewhereFilterActions(std::move(prewhere_actions_dag)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ffed19185d3..9042303d0e4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1412,7 +1412,7 @@ void Planner::buildPlanForQueryNode() checkStoragesSupportTransactions(planner_context); const auto & table_filters = planner_context->getGlobalPlannerContext()->filters_for_table_expressions; - if (!select_query_options.only_analyze && !table_filters.empty()) // && top_level) + if (!select_query_options.only_analyze && !table_filters.empty()) { for (auto & [table_node, table_expression_data] : planner_context->getTableExpressionNodeToData()) { @@ -1420,7 +1420,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(ActionsDAG::clone(&*filters.filter_actions)); + table_expression_data.setFilterActions(filters.filter_actions->clone()); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 3217d3461d3..d55e5e99f71 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -646,7 +646,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = ActionsDAG::clone(table_expression_data.getFilterActions()); + if (const auto & filter_actions = table_expression_data.getFilterActions()) + table_expression_query_info.filter_actions_dag = std::make_shared(filter_actions->clone()); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +777,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(prewhere_actions)); + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -829,7 +830,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info) { - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&row_policy_filter_info->actions)); + table_expression_data.setRowLevelFilterActions(row_policy_filter_info->actions.clone()); add_filter(*row_policy_filter_info, "Row-level security filter"); } diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 9723a00a356..1d04fac3dc3 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -211,32 +211,32 @@ public: is_merge_tree = is_merge_tree_value; } - const ActionsDAGPtr & getPrewhereFilterActions() const + const std::optional & getPrewhereFilterActions() const { return prewhere_filter_actions; } - void setRowLevelFilterActions(ActionsDAGPtr row_level_filter_actions_value) + void setRowLevelFilterActions(ActionsDAG row_level_filter_actions_value) { row_level_filter_actions = std::move(row_level_filter_actions_value); } - const ActionsDAGPtr & getRowLevelFilterActions() const + const std::optional & getRowLevelFilterActions() const { return row_level_filter_actions; } - void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value) + void setPrewhereFilterActions(ActionsDAG prewhere_filter_actions_value) { prewhere_filter_actions = std::move(prewhere_filter_actions_value); } - const ActionsDAGPtr & getFilterActions() const + const std::optional & getFilterActions() const { return filter_actions; } - void setFilterActions(ActionsDAGPtr filter_actions_value) + void setFilterActions(ActionsDAG filter_actions_value) { filter_actions = std::move(filter_actions_value); } @@ -289,16 +289,16 @@ private: ColumnIdentifierToColumnName column_identifier_to_column_name; /// Valid for table, table function - ActionsDAGPtr filter_actions; + std::optional filter_actions; /// Valid for table, table function PrewhereInfoPtr prewhere_info; /// Valid for table, table function - ActionsDAGPtr prewhere_filter_actions; + std::optional prewhere_filter_actions; /// Valid for table, table function - ActionsDAGPtr row_level_filter_actions; + std::optional row_level_filter_actions; /// Is storage remote bool is_remote = false; From fee7e22c1f60feb0a4c176355453caad18cd5bc1 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Sun, 14 Jul 2024 15:23:34 +0330 Subject: [PATCH 022/321] Changed the error code --- src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 4388864434e..87a44db573d 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -36,7 +36,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int QUERY_NOT_ALLOWED; + extern const int BAD_QUERY_PARAMETER; } namespace @@ -150,7 +150,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( } else if (!configuration->isPathWithGlobs()) { - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "ObjectStorageQueue url must either end with '/' or contain globs"); + throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs"); } checkAndAdjustSettings(*queue_settings, engine_args, mode > LoadingStrictnessLevel::CREATE, log); From 22b37d526bacfa281372211a81a3daf1518ca5d6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 15 Jul 2024 17:00:47 +0200 Subject: [PATCH 023/321] update joingGet and add joinGetOrNull --- .../functions/other-functions.md | 138 +++++++++++++++--- 1 file changed, 121 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..12d082fe0f3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2449,11 +2449,11 @@ As you can see, `runningAccumulate` merges states for each group of rows separat ## joinGet -The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). - -Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. +The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. +:::note Only supports tables created with the `ENGINE = Join(ANY, LEFT, )` statement. +::: **Syntax** @@ -2463,26 +2463,32 @@ joinGet(join_storage_table_name, `value_column`, join_keys) **Arguments** -- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. The identifier is searched in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. +- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. - `value_column` — name of the column of the table that contains required data. - `join_keys` — list of keys. +:::note +The identifier is searched for in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. +::: + **Returned value** -Returns a list of values corresponded to list of keys. - -If certain does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting. +- Returns a list of values corresponded to the list of keys. +:::note +If a certain key does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting during table creation. More info about `join_use_nulls` in [Join operation](../../engines/table-engines/special/join.md). +::: **Example** Input table: ```sql -CREATE DATABASE db_test -CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1 -INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) +CREATE DATABASE db_test; +CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id); +INSERT INTO db_test.id_val VALUES (1, 11)(2, 12)(4, 13); +SELECT * FROM db_test.id_val; ``` ```text @@ -2496,18 +2502,116 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) Query: ```sql -SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 +SELECT number, joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4); ``` Result: ```text -┌─joinGet(db_test.id_val, 'val', toUInt32(number))─┐ -│ 0 │ -│ 11 │ -│ 12 │ -│ 0 │ -└──────────────────────────────────────────────────┘ + ┌─number─┬─joinGet('db_test.id_val', 'val', toUInt32(number))─┐ +1. │ 0 │ 0 │ +2. │ 1 │ 11 │ +3. │ 2 │ 12 │ +4. │ 3 │ 0 │ + └────────┴────────────────────────────────────────────────────┘ +``` + +Setting `join_use_nulls` can be used during table creation to change the behaviour of what gets returned if no key exists in the source table. + +```sql +CREATE DATABASE db_test; +CREATE TABLE db_test.id_val_nulls(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls=1; +INSERT INTO db_test.id_val_nulls VALUES (1, 11)(2, 12)(4, 13); +SELECT * FROM db_test.id_val_nulls; +``` + +```text +┌─id─┬─val─┐ +│ 4 │ 13 │ +│ 2 │ 12 │ +│ 1 │ 11 │ +└────┴─────┘ +``` + +Query: + +```sql +SELECT number, joinGet(db_test.id_val_nulls, 'val', toUInt32(number)) from numbers(4); +``` + +Result: + +```text + ┌─number─┬─joinGet('db_test.id_val_nulls', 'val', toUInt32(number))─┐ +1. │ 0 │ ᴺᵁᴸᴸ │ +2. │ 1 │ 11 │ +3. │ 2 │ 12 │ +4. │ 3 │ ᴺᵁᴸᴸ │ + └────────┴──────────────────────────────────────────────────────────┘ +``` + +## joinGetOrNull + +Like [joinGet](#joinget) but returns `NULL` when the key is missing instead of returning the default value. + +**Syntax** + +```sql +joinGetOrNull(join_storage_table_name, `value_column`, join_keys) +``` + +**Arguments** + +- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. +- `value_column` — name of the column of the table that contains required data. +- `join_keys` — list of keys. + +:::note +The identifier is searched for in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. +::: + +**Returned value** + +- Returns a list of values corresponded to the list of keys. + +:::note +If a certain key does not exist in source table then `NULL` is returned for that key. +::: + +**Example** + +Input table: + +```sql +CREATE DATABASE db_test; +CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id); +INSERT INTO db_test.id_val VALUES (1, 11)(2, 12)(4, 13); +SELECT * FROM db_test.id_val; +``` + +```text +┌─id─┬─val─┐ +│ 4 │ 13 │ +│ 2 │ 12 │ +│ 1 │ 11 │ +└────┴─────┘ +``` + +Query: + +```sql +SELECT number, joinGetOrNull(db_test.id_val, 'val', toUInt32(number)) from numbers(4); +``` + +Result: + +```text + ┌─number─┬─joinGetOrNull('db_test.id_val', 'val', toUInt32(number))─┐ +1. │ 0 │ ᴺᵁᴸᴸ │ +2. │ 1 │ 11 │ +3. │ 2 │ 12 │ +4. │ 3 │ ᴺᵁᴸᴸ │ + └────────┴──────────────────────────────────────────────────────────┘ ``` ## catboostEvaluate From 9b6bdee5f3f80661577e0204f622dd6e41571806 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 16 Jul 2024 13:44:50 +0100 Subject: [PATCH 024/321] 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 025/321] 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 026/321] 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 7ea3324776bd4cb8cc886822a9b30d3dfcaff5a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 16:31:34 +0000 Subject: [PATCH 027/321] Refactor in VirtualColumnUtils --- src/Interpreters/ActionsDAG.cpp | 12 ++++----- src/Interpreters/ActionsDAG.h | 3 +-- .../useDataParallelAggregation.cpp | 8 +++--- src/Processors/QueryPlan/SortingStep.cpp | 10 +++++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +-- .../StorageObjectStorageSource.cpp | 15 ++++++++--- .../StorageObjectStorageSource.h | 2 +- src/Storages/StorageFile.cpp | 8 ++++-- src/Storages/StorageURL.cpp | 6 +++-- .../System/StorageSystemDetachedParts.cpp | 4 +-- .../StorageSystemDroppedTablesParts.cpp | 4 +-- .../System/StorageSystemDroppedTablesParts.h | 6 ++--- .../System/StorageSystemPartsBase.cpp | 12 ++++----- src/Storages/System/StorageSystemPartsBase.h | 6 ++--- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 27 +++++++++++-------- src/Storages/VirtualColumnUtils.h | 13 ++++----- 18 files changed, 87 insertions(+), 59 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4f03a9e1602..e001406408f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -624,9 +624,9 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) } } -ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) +ActionsDAG ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { - auto actions = std::make_unique(); + ActionsDAG actions; std::unordered_map copy_map; struct Frame @@ -661,21 +661,21 @@ ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool rem if (remove_aliases && frame.node->type == ActionType::ALIAS) copy_node = copy_map[frame.node->children.front()]; else - copy_node = &actions->nodes.emplace_back(*frame.node); + copy_node = &actions.nodes.emplace_back(*frame.node); if (frame.node->type == ActionType::INPUT) - actions->inputs.push_back(copy_node); + actions.inputs.push_back(copy_node); stack.pop(); } } - for (auto & node : actions->nodes) + for (auto & node : actions.nodes) for (auto & child : node.children) child = copy_map[child]; for (const auto * output : outputs) - actions->outputs.push_back(copy_map[output]); + actions.outputs.push_back(copy_map[output]); return actions; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 05948ccf928..6f5c3d3b0df 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,13 +261,12 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } static ActionsDAGPtr clone(const ActionsDAG * from); ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; ActionsDAG clone() const; - static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); + static ActionsDAG cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); /// Execute actions for header. Input block must have empty columns. /// Result should be equal to the execution of ExpressionActions built from this DAG. diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 7e0260c0040..0eeaec9bde7 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -74,11 +74,11 @@ void removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG::Node * nod /// Our objective is to replace injective function nodes in `actions` results with its children /// until only the irreducible subset of nodes remains. Against these set of nodes we will match partition key expression /// to determine if it maps all rows with the same value of group by key to the same partition. -NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAGPtr & actions) +NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG & actions) { NodeSet irreducible; NodeSet visited; - for (const auto & node : actions->getOutputs()) + for (const auto & node : actions.getOutputs()) removeInjectiveFunctionsFromResultsRecursively(node, irreducible, visited); return irreducible; } @@ -158,7 +158,7 @@ bool isPartitionKeySuitsGroupByKey( auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); - const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); + const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames(); const auto & partition_actions = reading.getStorageMetadata()->getPartitionKey().expression->getActionsDAG(); @@ -169,7 +169,7 @@ bool isPartitionKeySuitsGroupByKey( const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_key_actions); - const auto matches = matchTrees(group_by_key_actions->getOutputs(), partition_actions); + const auto matches = matchTrees(group_by_key_actions.getOutputs(), partition_actions); return allOutputsDependsOnlyOnAllowedNodes(partition_actions, irreducibe_nodes, matches); } diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 48fad9f5fdb..e8e761e7ab0 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -447,6 +447,13 @@ void SortingStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } + if (!partition_by_description.empty()) + { + settings.out << prefix << "Partition by description: "; + dumpSortDescription(partition_by_description, settings.out); + settings.out << '\n'; + } + if (limit) settings.out << prefix << "Limit " << limit << '\n'; } @@ -461,6 +468,9 @@ void SortingStep::describeActions(JSONBuilder::JSONMap & map) const else map.add("Sort Description", explainSortDescription(result_description)); + if (!partition_by_description.empty()) + map.add("Partition By Description", explainSortDescription(partition_by_description)); + if (limit) map.add("Limit", limit); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 878e0420665..9aa9490198a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1154,7 +1154,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (!virtual_columns_block.has(input->result_name)) valid = false; - PartitionPruner partition_pruner(metadata_snapshot, filter_dag.get(), local_context, true /* strict */); + PartitionPruner partition_pruner(metadata_snapshot, &*filter_dag, local_context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; @@ -1162,7 +1162,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (valid) { virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_dag), virtual_columns_block, local_context); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a6a40a808e5..a37dbfa554c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -465,7 +465,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( return; part_offset_condition.emplace(KeyCondition{ - dag.get(), + &*dag, context, sample.getNames(), std::make_shared(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), @@ -488,7 +488,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar return {}; auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(dag, virtual_columns_block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), virtual_columns_block, context); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index c86b56d3f1b..e760098f10f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -139,7 +139,10 @@ std::shared_ptr StorageObjectStorageSourc paths.reserve(keys.size()); for (const auto & key : keys) paths.push_back(fs::path(configuration->getNamespace()) / key); - VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); + + VirtualColumnUtils::buildSetsForDAG(*filter_dag, local_context); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(keys, paths, actions, virtual_columns); copy_configuration->setPaths(keys); } @@ -506,7 +509,11 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } recursive = key_with_globs == "/**"; - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns)) + { + VirtualColumnUtils::buildSetsForDAG(*filter_dag, getContext()); + filter_expr = std::make_shared(std::move(*filter_dag)); + } } else { @@ -570,14 +577,14 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne ++it; } - if (filter_dag) + if (filter_expr) { std::vector paths; paths.reserve(new_batch.size()); for (const auto & object_info : new_batch) paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_expr, virtual_columns); LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index b8418ddd07c..e466621e1e1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -208,7 +208,7 @@ private: ObjectInfos object_infos; ObjectInfos * read_keys; - ActionsDAGPtr filter_dag; + ExpressionActionsPtr filter_expr; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; std::vector expanded_keys; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c6acb358d89..fe6f494db00 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1130,12 +1130,16 @@ StorageFileSource::FilesIterator::FilesIterator( bool distributed_processing_) : WithContext(context_), files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_) { - ActionsDAGPtr filter_dag; + std::optional filter_dag; if (!distributed_processing && !archive_info && !files.empty()) filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); if (filter_dag) - VirtualColumnUtils::filterByPathOrFile(files, files, filter_dag, virtual_columns, context_); + { + VirtualColumnUtils::buildSetsForDAG(*filter_dag, context_); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(files, files, actions, virtual_columns); + } } String StorageFileSource::FilesIterator::next() diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9cec8c75ebe..c61bb8ac980 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -198,7 +198,7 @@ public: { uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses); - ActionsDAGPtr filter_dag; + std::optional filter_dag; if (!uris.empty()) filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); @@ -209,7 +209,9 @@ public: for (const auto & uri : uris) paths.push_back(Poco::URI(uri).getPath()); - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); + VirtualColumnUtils::buildSetsForDAG(*filter_dag, context); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(uris, paths, actions, virtual_columns); } } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 7e4c1de1c65..0d0ae666c10 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -307,7 +307,7 @@ protected: std::shared_ptr storage; std::vector columns_mask; - ActionsDAGPtr filter; + std::optional filter; const size_t max_block_size; const size_t num_streams; }; @@ -359,7 +359,7 @@ void StorageSystemDetachedParts::read( void ReadFromSystemDetachedParts::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto state = std::make_shared(StoragesInfoStream(nullptr, filter, context)); + auto state = std::make_shared(StoragesInfoStream({}, std::move(filter), context)); Pipe pipe; diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index c17d6402d88..defc4ec2d2a 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -11,7 +11,7 @@ namespace DB { -StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context) +StoragesDroppedInfoStream::StoragesDroppedInfoStream(std::optional filter, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -75,7 +75,7 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAGPtr & filte { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter) - VirtualColumnUtils::filterBlockWithDAG(filter, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter), block_to_filter, context); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.h b/src/Storages/System/StorageSystemDroppedTablesParts.h index dff9e41cce3..32468fc31b2 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.h +++ b/src/Storages/System/StorageSystemDroppedTablesParts.h @@ -9,7 +9,7 @@ namespace DB class StoragesDroppedInfoStream : public StoragesInfoStreamBase { public: - StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context); + StoragesDroppedInfoStream(std::optional filter, ContextPtr context); protected: bool tryLockTable(StoragesInfo &) override { @@ -30,9 +30,9 @@ public: std::string getName() const override { return "SystemDroppedTablesParts"; } protected: - std::unique_ptr getStoragesInfoStream(const ActionsDAGPtr &, const ActionsDAGPtr & filter, ContextPtr context) override + std::unique_ptr getStoragesInfoStream(std::optional, std::optional filter, ContextPtr context) override { - return std::make_unique(filter, context); + return std::make_unique(std::move(filter), context); } }; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index f7d1c1b3eb8..a0c9a5c61bd 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -91,7 +91,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state); } -StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context) +StoragesInfoStream::StoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -124,7 +124,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, /// Filter block_to_filter with column 'database'. if (filter_by_database) - VirtualColumnUtils::filterBlockWithDAG(filter_by_database, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_database), block_to_filter, context); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -204,7 +204,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter_by_other_columns) - VirtualColumnUtils::filterBlockWithDAG(filter_by_other_columns, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_other_columns), block_to_filter, context); rows = block_to_filter.rows(); } @@ -236,8 +236,8 @@ protected: std::shared_ptr storage; std::vector columns_mask; const bool has_state_column; - ActionsDAGPtr filter_by_database; - ActionsDAGPtr filter_by_other_columns; + std::optional filter_by_database; + std::optional filter_by_other_columns; }; ReadFromSystemPartsBase::ReadFromSystemPartsBase( @@ -318,7 +318,7 @@ void StorageSystemPartsBase::read( void ReadFromSystemPartsBase::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto stream = storage->getStoragesInfoStream(filter_by_database, filter_by_other_columns, context); + auto stream = storage->getStoragesInfoStream(std::move(filter_by_database), std::move(filter_by_other_columns), context); auto header = getOutputStream().header; MutableColumns res_columns = header.cloneEmptyColumns(); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 8671fd850f8..806af4a7bf8 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -116,7 +116,7 @@ protected: class StoragesInfoStream : public StoragesInfoStreamBase { public: - StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context); + StoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context); }; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. @@ -146,9 +146,9 @@ protected: StorageSystemPartsBase(const StorageID & table_id_, ColumnsDescription && columns); - virtual std::unique_ptr getStoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context) + virtual std::unique_ptr getStoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context) { - return std::make_unique(filter_by_database, filter_by_other_columns, context); + return std::make_unique(std::move(filter_by_database), std::move(filter_by_other_columns), context); } virtual void diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..85aaf4ad186 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -169,7 +169,7 @@ ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); return block.getByPosition(0).column; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1630d9fd9c4..32c6a558340 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,15 +77,20 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context) } } -void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) +void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context) +{ + buildSetsForDAG(dag, context); + auto actions = std::make_shared(std::move(dag)); + filterBlockWithExpression(actions, block); +} + +void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block) { - buildSetsForDAG(*dag, context); - auto actions = std::make_shared(std::move(*ActionsDAG::clone(dag))); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); /// Filter the block. - String filter_column_name = dag->getOutputs().at(0)->result_name; + String filter_column_name = actions->getActionsDAG().getOutputs().at(0)->result_name; ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst(); ConstantFilterDescription constant_filter(*filter_column); @@ -155,7 +160,7 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s block.getByName("_idx").column->assumeMutableRef().insert(idx); } -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) { if (!predicate || virtual_columns.empty()) return {}; @@ -171,7 +176,7 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con return splitFilterDagForAllowedInputs(predicate, &block); } -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) { Block block; for (const auto & column : virtual_columns) @@ -184,7 +189,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const for (size_t i = 0; i != paths.size(); ++i) addPathAndFileToVirtualColumns(block, paths[i], i); - filterBlockWithDAG(dag, block, context); + filterBlockWithExpression(actions, block); return block.getByName("_idx").column; } @@ -355,15 +360,15 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) { if (!predicate) - return nullptr; + return {}; ActionsDAG::Nodes additional_nodes; const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes); if (!res) - return nullptr; + return {}; return ActionsDAG::cloneSubDAG({res}, true); } @@ -372,7 +377,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, { auto dag = splitFilterDagForAllowedInputs(predicate, &block); if (dag) - filterBlockWithDAG(dag, block, context); + filterBlockWithDAG(std::move(*dag), block, context); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 208aa7a8100..72c45964ff4 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,8 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context); +void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context); +void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block); /// Builds sets used by ActionsDAG inplace. void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); @@ -32,7 +33,7 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); /// Extract from the input stream a set of `name` column values template @@ -49,14 +50,14 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns); -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context); +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns); template -void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) { - auto indexes_column = getFilterByPathAndFileIndexes(paths, dag, virtual_columns, context); + auto indexes_column = getFilterByPathAndFileIndexes(paths, actions, virtual_columns); const auto & indexes = typeid_cast(*indexes_column).getData(); if (indexes.size() == sources.size()) return; From 0954eefb076d36ec5804b46e594005cd7f4030bf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 17:01:35 +0000 Subject: [PATCH 028/321] Revert SortingStep changes. --- src/Processors/QueryPlan/SortingStep.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index e8e761e7ab0..48fad9f5fdb 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -447,13 +447,6 @@ void SortingStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } - if (!partition_by_description.empty()) - { - settings.out << prefix << "Partition by description: "; - dumpSortDescription(partition_by_description, settings.out); - settings.out << '\n'; - } - if (limit) settings.out << prefix << "Limit " << limit << '\n'; } @@ -468,9 +461,6 @@ void SortingStep::describeActions(JSONBuilder::JSONMap & map) const else map.add("Sort Description", explainSortDescription(result_description)); - if (!partition_by_description.empty()) - map.add("Partition By Description", explainSortDescription(partition_by_description)); - if (limit) map.add("Limit", limit); } From 2dbd04c8a77bf262f0965ddef9a2c166c22fcf55 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 17 Jul 2024 17:01:22 +0200 Subject: [PATCH 029/321] add toIntXYZ documentation --- .../functions/type-conversion-functions.md | 1275 ++++++++++++++++- 1 file changed, 1239 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 61e84ca72d1..057083d317f 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -49,105 +49,1308 @@ SETTINGS cast_keep_nullable = 1 └──────────────────┴─────────────────────┴──────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256) +## toInt8 -Converts an input value to a value the [Int](../data-types/int-uint.md) data type. This function family includes: +Converts an input value to a value of type `Int8`. -- `toInt8(expr)` — Converts to a value of data type `Int8`. -- `toInt16(expr)` — Converts to a value of data type `Int16`. -- `toInt32(expr)` — Converts to a value of data type `Int32`. -- `toInt64(expr)` — Converts to a value of data type `Int64`. -- `toInt128(expr)` — Converts to a value of data type `Int128`. -- `toInt256(expr)` — Converts to a value of data type `Int256`. +**Syntax** + +```sql +toInt8(expr) +``` **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: **Returned value** -Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` data type. +- 8-bit integer value. [Int8](../data-types/int-uint.md). -Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: -The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt8(-8), + toInt8(-8.8), + toInt8('-8'); +``` + +Result: + +```response + ┌─toInt8(-8)─┬─toInt8(-8.8)─┬─toInt8('-8')─┐ +1. │ -8 │ -8 │ -8 │ + └────────────┴──────────────┴──────────────┘ +``` + +**See also** + +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrNull`](#toint8ornull). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrZero + +Like [`toInt8`](#toint8), it takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt8OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise `0`. [Int8](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); +SELECT + toInt8OrZero('-8'), + toInt8OrZero('abc'); ``` Result: ```response -┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ -│ -9223372036854775808 │ 32 │ 16 │ 8 │ -└──────────────────────┴─────────────┴───────────────┴─────────────┘ + ┌─toInt8OrZero('-8')─┬─toInt8OrZero('abc')─┐ +1. │ -8 │ 0 │ + └────────────────────┴─────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrZero +**See also** -Takes an argument of type [String](../data-types/string.md) and tries to parse it into an Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `0`. +- [`toInt8`](#toint8). +- [`toInt8OrNull`](#toint8ornull). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrNull + +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt8OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise `NULL`. [Int8](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrZero('123123'), toInt8OrZero('123qwe123'); +SELECT toInt8OrNull('-8'), toInt8OrNull('abc'); ``` Result: ```response -┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ -│ 123123 │ 0 │ -└─────────────────────────┴───────────────────────────┘ + ┌─toInt8OrNull('-8')─┬─toInt8OrNull('abc')─┐ +1. │ -8 │ ᴺᵁᴸᴸ │ + └────────────────────┴─────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrNull +**See also** -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `NULL`. +- [`toInt8`](#toint8). +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrDefault + +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt8OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise returns the default value. [Int8](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); +SELECT + toInt8OrDefault('-8', CAST('-1', 'Int8')), + toInt8OrDefault('abc', CAST('-1', 'Int8')); ``` Result: ```response -┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ -│ 123123 │ ᴺᵁᴸᴸ │ -└─────────────────────────┴───────────────────────────┘ + ┌─toInt8OrDefault('-8', CAST('-1', 'Int8'))─┬─toInt8OrDefault('abc', CAST('-1', 'Int8'))─┐ +1. │ -8 │ -1 │ + └───────────────────────────────────────────┴────────────────────────────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrDefault +**See also** -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns the default type value. +- [`toInt8`](#toint8). +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrNull`](#toint8orNull). + +## toInt16 + +Converts an input value to a value of type `Int16`. + +**Syntax** + +```sql +toInt16(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value. [Int16](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt16(-16), + toInt16(-16.16), + toInt16('-16'); +``` + +Result: + +```response + ┌─toInt16(-16)─┬─toInt16(-16.16)─┬─toInt16('-16')─┐ +1. │ -16 │ -16 │ -16 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrNull`](#toint16ornull). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrZero + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt16OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise `0`. [Int16](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrDefault('123123', cast('-1' as Int64)), toInt8OrDefault('123qwe123', cast('-1' as Int8)); +SELECT + toInt16OrZero('-16'), + toInt16OrZero('abc'); ``` Result: ```response -┌─toInt64OrDefault('123123', CAST('-1', 'Int64'))─┬─toInt8OrDefault('123qwe123', CAST('-1', 'Int8'))─┐ -│ 123123 │ -1 │ -└─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ + ┌─toInt16OrZero('-16')─┬─toInt16OrZero('abc')─┐ +1. │ -16 │ 0 │ + └──────────────────────┴──────────────────────┘ ``` +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrNull`](#toint16ornull). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrNull + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt16OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise `NULL`. [Int16](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt16OrNull('-16'), + toInt16OrNull('abc'); +``` + +Result: + +```response + ┌─toInt16OrNull('-16')─┬─toInt16OrNull('abc')─┐ +1. │ -16 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrDefault + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt16OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise returns the default value. [Int16](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt16OrDefault('-16', cast('-1' as Int16)), toInt16OrDefault('abc', cast('-1' as Int16)); +``` + +Result: + +```response + ┌─toInt16OrDefault('-16', CAST('-1', 'Int16'))─┬─toInt16OrDefault('abc', CAST('-1', 'Int16'))─┐ +1. │ -16 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrNull`](#toint16ornull). + +## toInt32 + +Converts an input value to a value of type `Int32`. + +**Syntax** + +```sql +toInt32(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value. [Int32](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt32(-32), + toInt32(-32.32), + toInt32('-32') +``` + +Result: + +```response + ┌─toInt32(-32)─┬─toInt32(-32.32)─┬─toInt32('-32')─┐ +1. │ -32 │ -32 │ -32 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrNull`](#toint32ornull). +- [`toInt32OrDefault`](#toint32ordefault). + +## toInt32OrZero + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt32OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise `0`. [Int32](../data-types/int-uint.md) + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrZero('-32'), toInt32OrZero('abc'); +``` + +Result: + +```response + ┌─toInt32OrZero('-32')─┬─toInt32OrZero('abc')─┐ +1. │ -32 │ 0 │ + └──────────────────────┴──────────────────────┘ +``` +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrNull`](#toint32ornull). +- [`toInt32OrDefault`](#toint32ordefault). +- +## toInt32OrNull + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt32OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise `NULL`. [Int32](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrNull('-32'), toInt32OrNull('abc'); +``` + +Result: + +```response + ┌─toInt32OrNull('-32')─┬─toInt32OrNull('abc')─┐ +1. │ -32 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrDefault`](#toint32ordefault). + +## toInt32OrDefault + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt32OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise returns the default value. [Int32](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. + ::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrDefault('-32', cast('-1' as Int32)), toInt32OrDefault('abc', cast('-1' as Int32)); +``` + +Result: + +```response + ┌─toInt32OrDefault('-32', CAST('-1', 'Int32'))─┬─toInt32OrDefault('abc', CAST('-1', 'Int32'))─┐ +1. │ -32 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrNull`](#toint32ornull). + +## toInt64 + +Converts an input value to a value of type `Int64`. + +**Syntax** + +```sql +toInt64(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt64(-64), + toInt64(-64.64), + toInt64('-64'); +``` + +Result: + +```response + ┌─toInt64(-64)─┬─toInt64(-64.64)─┬─toInt64('-64')─┐ +1. │ -64 │ -64 │ -64 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrNull`](#toint64ornull). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrZero + +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt64OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 64-bit integer value if successful, otherwise `0`. [Int64](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrZero('-64'), + toInt64OrZero('abc'); +``` + +Result: + +```response + ┌─toInt64OrZero('-64')─┬─toInt64OrZero('abc')─┐ +1. │ -64 │ 0 │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrNull`](#toint64ornull). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrNull + +Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt64OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrNull('-64'), + toInt64OrNull('abc'); +``` + +Result: + +```response + ┌─toInt64OrNull('-64')─┬─toInt64OrNull('abc')─┐ +1. │ -64 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrDefault + +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt64OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. + ::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrDefault('-64', CAST('-1', 'Int64')), + toInt64OrDefault('abc', CAST('-1', 'Int64')); +``` + +Result: + +```response + ┌─toInt64OrDefault('-64', CAST('-1', 'Int64'))─┬─toInt64OrDefault('abc', CAST('-1', 'Int64'))─┐ +1. │ -64 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrNull`](#toint64ornull). + +## toInt128 + +Converts an input value to a value of type `Int128`. + +**Syntax** + +```sql +toInt128(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value. [Int128](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt128(-128), + toInt128(-128.8), + toInt128('-128'), +``` + +Result: + +```response + ┌─toInt128(-128)─┬─toInt128(-128.8)─┬─toInt128('-128')─┐ +1. │ -128 │ -128 │ -128 │ + └────────────────┴──────────────────┴──────────────────┘ +``` + +**See also** + +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrNull`](#toint128ornull). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrZero + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt128OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise `0`. [Int128](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrZero('-128'), + toInt128OrZero('abc'); +``` + +Result: + +```response + ┌─toInt128OrZero('-128')─┬─toInt128OrZero('abc')─┐ +1. │ -128 │ 0 │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrNull`](#toint128ornull). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrNull + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt128OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise `NULL`. [Int128](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrNull('-128'), + toInt128OrNull('abc'); +``` + +Result: + +```response + ┌─toInt128OrNull('-128')─┬─toInt128OrNull('abc')─┐ +1. │ -128 │ ᴺᵁᴸᴸ │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrDefault + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt128OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise returns the default value. [Int128](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrDefault('-128', CAST('-1', 'Int128')), + toInt128OrDefault('abc', CAST('-1', 'Int128')); +``` + +Result: + +```response + ┌─toInt128OrDefault('-128', CAST('-1', 'Int128'))─┬─toInt128OrDefault('abc', CAST('-1', 'Int128'))─┐ +1. │ -128 │ -1 │ + └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrNull`](#toint128ornull). + +## toInt256 + +Converts an input value to a value of type `Int256`. + +**Syntax** + +```sql +toInt256(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value. [Int256](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt256(-256), + toInt256(-256.256), + toInt256('-256'); +``` + +Result: + +```response + ┌─toInt256(-256)─┬─toInt256(-256.256)─┬─toInt256('-256')─┐ +1. │ -256 │ -256 │ -256 │ + └────────────────┴────────────────────┴──────────────────┘ +``` + +**See also** + +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrNull`](#toint256ornull). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrZero + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt256OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise `0`. [Int256](../data-types/int-uint.md). + +:::note +Functions uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrZero('-256'), + toInt256OrZero('abc'); +``` + +Result: + +```response + ┌─toInt256OrZero('-256')─┬─toInt256OrZero('abc')─┐ +1. │ -256 │ 0 │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrNull`](#toint256ornull). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrNull + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt256OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise `NULL`. [Int256](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrNull('-256'), + toInt256OrNull('abc'); +``` + +Result: + +```response + ┌─toInt256OrNull('-256')─┬─toInt256OrNull('abc')─┐ +1. │ -256 │ ᴺᵁᴸᴸ │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrDefault + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt256OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise returns the default value. [Int256](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrDefault('-256', CAST('-1', 'Int256')), + toInt256OrDefault('abc', CAST('-1', 'Int256')); +``` + +Result: + +```response + ┌─toInt256OrDefault('-256', CAST('-1', 'Int256'))─┬─toInt256OrDefault('abc', CAST('-1', 'Int256'))─┐ +1. │ -256 │ -1 │ + └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrNull`](#toint256ornull). ## toUInt(8\|16\|32\|64\|256) @@ -167,7 +1370,7 @@ Converts an input value to the [UInt](../data-types/int-uint.md) data type. This - Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type. -Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. +Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. @@ -2289,7 +3492,7 @@ Result: └─────────────────────┴─────────────────┴─────────────────────────────────────┘ ``` -**See Also** +**See also** - [RFC 1123](https://datatracker.ietf.org/doc/html/rfc1123) - [toDate](#todate) From 80e1377e5d2223176274c319938187f0da799280 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 17:30:28 +0000 Subject: [PATCH 030/321] Fixing build. --- src/Storages/System/StorageSystemTables.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 43b761d84b1..d6b577bf6c8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -125,7 +125,7 @@ ColumnPtr getFilteredTables( block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); return block.getByPosition(0).column; } From 55355f43ad420456467121ce43072a10791c5cc8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:19:58 +0200 Subject: [PATCH 031/321] Fix bad code: it was catching exceptions --- src/IO/WithFileSize.cpp | 48 +++++++++---------- ...ry_and_native_with_binary_encoded_types.sh | 4 +- 2 files changed, 25 insertions(+), 27 deletions(-) diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 3660d962c08..8cea12fa200 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -14,40 +14,38 @@ namespace ErrorCodes } template -static size_t getFileSize(T & in) +static std::optional tryGetFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - { return with_file_size->getFileSize(); - } + + return std::nullopt; +} + +template +static size_t getFileSize(T & in) +{ + if (auto maybe_size = tryGetFileSize(in)) + return *maybe_size; throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } -size_t getFileSizeFromReadBuffer(ReadBuffer & in) -{ - if (auto * delegate = dynamic_cast(&in)) - { - return getFileSize(delegate->getWrappedReadBuffer()); - } - else if (auto * compressed = dynamic_cast(&in)) - { - return getFileSize(compressed->getWrappedReadBuffer()); - } - - return getFileSize(in); -} - std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in) { - try - { - return getFileSizeFromReadBuffer(in); - } - catch (...) - { - return std::nullopt; - } + if (auto * delegate = dynamic_cast(&in)) + return tryGetFileSize(delegate->getWrappedReadBuffer()); + else if (auto * compressed = dynamic_cast(&in)) + return tryGetFileSize(compressed->getWrappedReadBuffer()); + return tryGetFileSize(in); +} + +size_t getFileSizeFromReadBuffer(ReadBuffer & in) +{ + if (auto maybe_size = tryGetFileSizeFromReadBuffer(in)) + return *maybe_size; + + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } bool isBufferWithFileSize(const ReadBuffer & in) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index 723b11ad620..0c585d36348 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test { - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" } test "materialize(42)::UInt8" From e0aedb992f647a8dcd226bc8775795ecad91a551 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:34:04 +0200 Subject: [PATCH 032/321] Add a test --- .../03206_no_exceptions_clickhouse_local.reference | 1 + .../0_stateless/03206_no_exceptions_clickhouse_local.sh | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.reference create mode 100755 tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh diff --git a/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.reference b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.reference new file mode 100644 index 00000000000..11277a62b06 --- /dev/null +++ b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.reference @@ -0,0 +1 @@ +Hello world diff --git a/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh new file mode 100755 index 00000000000..86839a228dc --- /dev/null +++ b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so the grpc library is not built + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +CLICKHOUSE_TERMINATE_ON_ANY_EXCEPTION=1 ${CLICKHOUSE_LOCAL} --query "SELECT * FROM table" --input-format CSV <<<"Hello, world" From c7be25f0a167c2c5ab6944b47779be2f90af443d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 04:54:36 +0200 Subject: [PATCH 033/321] Fix everything --- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 2 +- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 +- src/IO/Archives/LibArchiveReader.cpp | 2 +- src/IO/Archives/ZipArchiveReader.cpp | 2 +- src/IO/AsynchronousReadBufferFromFileDescriptor.cpp | 2 +- src/IO/AsynchronousReadBufferFromFileDescriptor.h | 2 +- src/IO/ConcatSeekableReadBuffer.h | 2 +- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 +- src/IO/MMapReadBufferFromFileDescriptor.h | 2 +- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/ParallelReadBuffer.h | 2 +- src/IO/ReadBufferFromEmptyFile.h | 2 +- src/IO/ReadBufferFromEncryptedFile.h | 2 +- src/IO/ReadBufferFromFileBase.cpp | 6 ++---- src/IO/ReadBufferFromFileBase.h | 2 +- src/IO/ReadBufferFromFileDecorator.cpp | 4 ++-- src/IO/ReadBufferFromFileDecorator.h | 2 +- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadBufferFromS3.cpp | 6 +++--- src/IO/ReadBufferFromS3.h | 2 +- src/IO/ReadWriteBufferFromHTTP.cpp | 7 ++----- src/IO/ReadWriteBufferFromHTTP.h | 2 +- src/IO/WithFileSize.cpp | 10 +++++++++- src/IO/WithFileSize.h | 7 ++++--- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 4 ++-- .../HDFS/AsynchronousReadBufferFromHDFS.h | 2 +- src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 8 ++++---- src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h | 2 +- 32 files changed, 52 insertions(+), 48 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index 9a802348998..3dc8fcc39cb 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -34,7 +34,7 @@ public: String getFileName() const override { return impl->getFileName(); } - size_t getFileSize() override { return impl->getFileSize(); } + std::optional tryGetFileSize() override { return impl->tryGetFileSize(); } String getInfoForLog() override { return impl->getInfoForLog(); } diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index da1ea65f2ea..a36a8b031b4 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -253,7 +253,7 @@ void ReadBufferFromAzureBlobStorage::initialize() initialized = true; } -size_t ReadBufferFromAzureBlobStorage::getFileSize() +std::optional ReadBufferFromAzureBlobStorage::tryGetFileSize() { if (!blob_client) blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index d328195cc26..f407f27e099 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -42,7 +42,7 @@ public: bool supportsRightBoundedReads() const override { return true; } - size_t getFileSize() override; + std::optional tryGetFileSize() override; size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const override; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index e36365a8174..9f1cb681f1a 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -41,7 +41,7 @@ public: void setReadUntilEnd() override { setReadUntilPosition(getFileSize()); } - size_t getFileSize() override { return getTotalSize(blobs_to_read); } + std::optional tryGetFileSize() override { return getTotalSize(blobs_to_read); } size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index e3fe63fa40d..31bad4d6638 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -321,7 +321,7 @@ public: off_t getPosition() override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } String getFileName() const override { return handle.getFileName(); } - size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } + std::optional tryGetFileSize() override { return handle.getFileInfo().uncompressed_size; } Handle releaseHandle() && { return std::move(handle); } diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 2a9b7a43519..12b07d550c2 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -317,7 +317,7 @@ public: String getFileName() const override { return handle.getFileName(); } - size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } + std::optional tryGetFileSize() override { return handle.getFileInfo().uncompressed_size; } /// Releases owned handle to pass it to an enumerator. HandleHolder releaseHandle() && diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index f8c00d62732..6c4bd09b76f 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -244,7 +244,7 @@ void AsynchronousReadBufferFromFileDescriptor::rewind() file_offset_of_buffer_end = 0; } -size_t AsynchronousReadBufferFromFileDescriptor::getFileSize() +std::optional AsynchronousReadBufferFromFileDescriptor::tryGetFileSize() { return getSizeFromFileDescriptor(fd, getFileName()); } diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index 82659b1aca7..097979fbe00 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -68,7 +68,7 @@ public: /// Seek to the beginning, discarding already read data if any. Useful to reread file that changes on every read. void rewind(); - size_t getFileSize() override; + std::optional tryGetFileSize() override; size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } diff --git a/src/IO/ConcatSeekableReadBuffer.h b/src/IO/ConcatSeekableReadBuffer.h index c8c16c5d887..609f0dc25b8 100644 --- a/src/IO/ConcatSeekableReadBuffer.h +++ b/src/IO/ConcatSeekableReadBuffer.h @@ -21,7 +21,7 @@ public: off_t seek(off_t off, int whence) override; off_t getPosition() override; - size_t getFileSize() override { return total_size; } + std::optional tryGetFileSize() override { return total_size; } private: bool nextImpl() override; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index f27828f71b2..83dd192de54 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -87,7 +87,7 @@ off_t MMapReadBufferFromFileDescriptor::seek(off_t offset, int whence) return new_pos; } -size_t MMapReadBufferFromFileDescriptor::getFileSize() +std::optional MMapReadBufferFromFileDescriptor::tryGetFileSize() { return getSizeFromFileDescriptor(getFD(), getFileName()); } diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index f774538374a..de44ec3f9d8 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -38,7 +38,7 @@ public: int getFD() const; - size_t getFileSize() override; + std::optional tryGetFileSize() override; size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) const override; bool supportsReadAt() override { return true; } diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index e6771235a8e..89cff670e37 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -152,7 +152,7 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) return offset; } -size_t ParallelReadBuffer::getFileSize() +std::optional ParallelReadBuffer::tryGetFileSize() { return file_size; } diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index cfeec2b3677..8852472a8bc 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -33,7 +33,7 @@ public: ~ParallelReadBuffer() override { finishAndWait(); } off_t seek(off_t off, int whence) override; - size_t getFileSize() override; + std::optional tryGetFileSize() override; off_t getPosition() override; const SeekableReadBuffer & getReadBuffer() const { return input; } diff --git a/src/IO/ReadBufferFromEmptyFile.h b/src/IO/ReadBufferFromEmptyFile.h index f21f2f507dc..b15299dafee 100644 --- a/src/IO/ReadBufferFromEmptyFile.h +++ b/src/IO/ReadBufferFromEmptyFile.h @@ -19,7 +19,7 @@ private: std::string getFileName() const override { return ""; } off_t seek(off_t /*off*/, int /*whence*/) override { return 0; } off_t getPosition() override { return 0; } - size_t getFileSize() override { return 0; } + std::optional tryGetFileSize() override { return 0; } }; } diff --git a/src/IO/ReadBufferFromEncryptedFile.h b/src/IO/ReadBufferFromEncryptedFile.h index 3626daccb3e..213d242bb91 100644 --- a/src/IO/ReadBufferFromEncryptedFile.h +++ b/src/IO/ReadBufferFromEncryptedFile.h @@ -30,7 +30,7 @@ public: void setReadUntilEnd() override { in->setReadUntilEnd(); } - size_t getFileSize() override { return in->getFileSize(); } + std::optional tryGetFileSize() override { return in->tryGetFileSize(); } private: bool nextImpl() override; diff --git a/src/IO/ReadBufferFromFileBase.cpp b/src/IO/ReadBufferFromFileBase.cpp index 4ac3f984f78..d42b12ba49b 100644 --- a/src/IO/ReadBufferFromFileBase.cpp +++ b/src/IO/ReadBufferFromFileBase.cpp @@ -26,11 +26,9 @@ ReadBufferFromFileBase::ReadBufferFromFileBase( ReadBufferFromFileBase::~ReadBufferFromFileBase() = default; -size_t ReadBufferFromFileBase::getFileSize() +std::optional ReadBufferFromFileBase::tryGetFileSize() { - if (file_size) - return *file_size; - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for read buffer"); + return file_size; } void ReadBufferFromFileBase::setProgressCallback(ContextPtr context) diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index 9870d8bbe43..c98dcd5a93e 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -50,7 +50,7 @@ public: clock_type = clock_type_; } - size_t getFileSize() override; + std::optional tryGetFileSize() override; void setProgressCallback(ContextPtr context); diff --git a/src/IO/ReadBufferFromFileDecorator.cpp b/src/IO/ReadBufferFromFileDecorator.cpp index 9ac0fb4e475..8a6468b9bd0 100644 --- a/src/IO/ReadBufferFromFileDecorator.cpp +++ b/src/IO/ReadBufferFromFileDecorator.cpp @@ -52,9 +52,9 @@ bool ReadBufferFromFileDecorator::nextImpl() return result; } -size_t ReadBufferFromFileDecorator::getFileSize() +std::optional ReadBufferFromFileDecorator::tryGetFileSize() { - return getFileSizeFromReadBuffer(*impl); + return tryGetFileSizeFromReadBuffer(*impl); } } diff --git a/src/IO/ReadBufferFromFileDecorator.h b/src/IO/ReadBufferFromFileDecorator.h index 6e62c7f741b..69f029c5cf7 100644 --- a/src/IO/ReadBufferFromFileDecorator.h +++ b/src/IO/ReadBufferFromFileDecorator.h @@ -27,7 +27,7 @@ public: ReadBuffer & getWrappedReadBuffer() { return *impl; } - size_t getFileSize() override; + std::optional tryGetFileSize() override; protected: std::unique_ptr impl; diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 76a80f145e7..51a1a5d8d93 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -253,7 +253,7 @@ void ReadBufferFromFileDescriptor::rewind() file_offset_of_buffer_end = 0; } -size_t ReadBufferFromFileDescriptor::getFileSize() +std::optional ReadBufferFromFileDescriptor::tryGetFileSize() { return getSizeFromFileDescriptor(fd, getFileName()); } diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index db256ef91c7..6083e744c95 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -69,7 +69,7 @@ public: /// Seek to the beginning, discarding already read data if any. Useful to reread file that changes on every read. void rewind(); - size_t getFileSize() override; + std::optional tryGetFileSize() override; bool checkIfActuallySeekable() override; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 9e001232e65..94f317802e3 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -313,15 +313,15 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence) return offset; } -size_t ReadBufferFromS3::getFileSize() +std::optional ReadBufferFromS3::tryGetFileSize() { if (file_size) - return *file_size; + return file_size; auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id); file_size = object_size; - return *file_size; + return file_size; } off_t ReadBufferFromS3::getPosition() diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index c6625c2d632..ff04f78ce7b 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -63,7 +63,7 @@ public: off_t getPosition() override; - size_t getFileSize() override; + std::optional tryGetFileSize() override; void setReadUntilPosition(size_t position) override; void setReadUntilEnd() override; diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index b753e66da48..2a62b11aa44 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -121,15 +121,12 @@ void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, s credentials.authenticate(request); } -size_t ReadWriteBufferFromHTTP::getFileSize() +std::optional ReadWriteBufferFromHTTP::tryGetFileSize() { if (!file_info) file_info = getFileInfo(); - if (file_info->file_size) - return *file_info->file_size; - - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", initial_uri.toString()); + return file_info->file_size; } bool ReadWriteBufferFromHTTP::supportsReadAt() diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index f496fe3ddcd..1c9bda53008 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -118,7 +118,7 @@ private: std::unique_ptr initialize(); - size_t getFileSize() override; + std::optional tryGetFileSize() override; bool supportsReadAt() override; diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 8cea12fa200..cbbcab83de2 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -13,11 +13,19 @@ namespace ErrorCodes extern const int UNKNOWN_FILE_SIZE; } +size_t WithFileSize::getFileSize() +{ + if (auto maybe_size = tryGetFileSize()) + return *maybe_size; + + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); +} + template static std::optional tryGetFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - return with_file_size->getFileSize(); + return with_file_size->tryGetFileSize(); return std::nullopt; } diff --git a/src/IO/WithFileSize.h b/src/IO/WithFileSize.h index 0ae3af98ea0..e5dc383fab0 100644 --- a/src/IO/WithFileSize.h +++ b/src/IO/WithFileSize.h @@ -10,15 +10,16 @@ class ReadBuffer; class WithFileSize { public: - virtual size_t getFileSize() = 0; + /// Returns nullopt if couldn't find out file size; + virtual std::optional tryGetFileSize() = 0; virtual ~WithFileSize() = default; + + size_t getFileSize(); }; bool isBufferWithFileSize(const ReadBuffer & in); size_t getFileSizeFromReadBuffer(ReadBuffer & in); - -/// Return nullopt if couldn't find out file size; std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in); size_t getDataOffsetMaybeCompressed(const ReadBuffer & in); diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 4c8c7974005..3b4eff28307 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -53,7 +53,7 @@ public: bool nextImpl() override; off_t seek(off_t off, int whence) override; off_t getPosition() override; - size_t getFileSize() override { return remote_file_size; } + std::optional tryGetFileSize() override { return remote_file_size; } private: std::unique_ptr local_file_holder; diff --git a/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp index 21df7e35284..3bbc4e8a2ea 100644 --- a/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -91,9 +91,9 @@ void AsynchronousReadBufferFromHDFS::prefetch(Priority priority) } -size_t AsynchronousReadBufferFromHDFS::getFileSize() +std::optional AsynchronousReadBufferFromHDFS::tryGetFileSize() { - return impl->getFileSize(); + return impl->tryGetFileSize(); } String AsynchronousReadBufferFromHDFS::getFileName() const diff --git a/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h index 5aef92315a4..9846d74453b 100644 --- a/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h @@ -35,7 +35,7 @@ public: void prefetch(Priority priority) override; - size_t getFileSize() override; + std::optional tryGetFileSize() override; String getFileName() const override; diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index be339d021dc..bf6f9db722c 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes } -struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory +struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory, public WithFileSize { String hdfs_uri; String hdfs_file_path; @@ -90,7 +90,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory tryGetFileSize() override { return file_size; } @@ -191,9 +191,9 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; -size_t ReadBufferFromHDFS::getFileSize() +std::optional ReadBufferFromHDFS::tryGetFileSize() { - return impl->getFileSize(); + return impl->tryGetFileSize(); } bool ReadBufferFromHDFS::nextImpl() diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h index d9671e7e445..5363f07967b 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h @@ -40,7 +40,7 @@ public: off_t getPosition() override; - size_t getFileSize() override; + std::optional tryGetFileSize() override; size_t getFileOffsetOfBufferEnd() const override; From cda846339be22c66cd0d35d49273a314fa3bdf69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 15:12:32 +0000 Subject: [PATCH 034/321] Remove ActionsDAG::clone --- src/Interpreters/ActionsDAG.cpp | 8 ---- src/Interpreters/ActionsDAG.h | 3 -- src/Interpreters/ExpressionAnalyzer.cpp | 9 +---- src/Interpreters/InterpreterSelectQuery.cpp | 24 ++++++------ src/Interpreters/MutationsInterpreter.cpp | 4 +- src/Planner/Planner.cpp | 38 +++++++++---------- src/Planner/PlannerExpressionAnalysis.h | 16 ++++---- src/Planner/PlannerJoins.cpp | 4 +- src/Processors/QueryPlan/ExpressionStep.cpp | 4 +- src/Processors/QueryPlan/FilterStep.cpp | 4 +- .../Optimizations/distinctReadInOrder.cpp | 10 ++--- .../Optimizations/filterPushDown.cpp | 4 +- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +-- .../Optimizations/optimizeReadInOrder.cpp | 28 +++++++------- .../optimizeUseAggregateProjection.cpp | 4 +- .../Optimizations/projectionsCommon.cpp | 7 ++-- .../Optimizations/removeRedundantDistinct.cpp | 20 +++++----- .../QueryPlan/ReadFromMergeTree.cpp | 16 ++++---- .../QueryPlan/SourceStepWithFilter.cpp | 8 ++-- src/Processors/QueryPlan/TotalsHavingStep.cpp | 22 +++++++---- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- src/Storages/SelectQueryInfo.h | 4 +- src/Storages/StorageBuffer.cpp | 10 +++-- src/Storages/StorageMerge.cpp | 8 ++-- src/Storages/VirtualColumnUtils.cpp | 2 +- 28 files changed, 135 insertions(+), 140 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e001406408f..53e04f24829 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1246,14 +1246,6 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) return true; } -ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) -{ - std::unordered_map old_to_new_nodes; - if (from == nullptr) - return nullptr; - return std::make_unique(from->clone(old_to_new_nodes)); -} - ActionsDAG ActionsDAG::clone() const { std::unordered_map old_to_new_nodes; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 6f5c3d3b0df..6f6c3f9bccb 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,8 +261,6 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr clone(const ActionsDAG * from); - ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; ActionsDAG clone() const; @@ -491,7 +489,6 @@ public: const ActionsDAG::Node * find(const String & output_name); private: - //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 0f350602777..6b5b129085d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,10 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - ActionsDAG dag = std::move(*ActionsDAG::clone(&prewhere_dag_and_flags->dag)); - ExpressionActions( - std::move(dag), - ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); + prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1979,9 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - ExpressionActions( - std::move(*ActionsDAG::clone(&before_where->dag)), - ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); + before_where->dag.updateHeader(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c85eb8310dc..e0073a6af5d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1501,7 +1501,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), + expressions.filter_info->actions.clone(), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1515,7 +1515,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.prewhere_info->row_level_filter)), + expressions.prewhere_info->row_level_filter->clone(), expressions.prewhere_info->row_level_column_name, true); @@ -1525,7 +1525,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.prewhere_info->prewhere_actions)), + expressions.prewhere_info->prewhere_actions->clone(), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -1627,7 +1627,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), + expressions.filter_info->actions.clone(), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -2056,20 +2056,22 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & prewhere_info = *query_info.prewhere_info; + auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); if (prewhere_info.row_level_filter) { pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.row_level_filter))), + row_level_actions, prewhere_info.row_level_column_name, true); }); } + auto filter_actions = std::make_shared(prewhere_info.prewhere_actions->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.prewhere_actions))), + header, filter_actions, prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2589,7 +2591,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&*alias_actions))); + auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), alias_actions->clone()); table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } @@ -2597,7 +2599,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2771,7 +2773,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2789,7 +2791,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( std::optional dag; if (expression) { - dag = std::move(*ActionsDAG::clone(&expression->dag)); + dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2838,7 +2840,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - ActionsDAG dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 5b710149d85..57ad5caa4c7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1281,7 +1281,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. @@ -1289,7 +1289,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } else { - auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e087c3691b4..fb721069e6e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -333,11 +333,11 @@ public: }; void addExpressionStep(QueryPlan & query_plan, - const ActionsAndProjectInputsFlagPtr & expression_actions, + ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = std::move(*ActionsDAG::clone(&expression_actions->dag)); + auto actions = std::move(expression_actions->dag); if (expression_actions->project_input) actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -348,11 +348,11 @@ void addExpressionStep(QueryPlan & query_plan, } void addFilterStep(QueryPlan & query_plan, - const FilterAnalysisResult & filter_analysis_result, + FilterAnalysisResult & filter_analysis_result, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = std::move(*ActionsDAG::clone(&filter_analysis_result.filter_actions->dag)); + auto actions = std::move(filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -544,7 +544,7 @@ void addMergingAggregatedStep(QueryPlan & query_plan, } void addTotalsHavingStep(QueryPlan & query_plan, - const PlannerExpressionsAnalysisResult & expression_analysis_result, + PlannerExpressionsAnalysisResult & expression_analysis_result, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, @@ -553,14 +553,14 @@ void addTotalsHavingStep(QueryPlan & query_plan, const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); - const auto & having_analysis_result = expression_analysis_result.getHaving(); + auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); + auto & having_analysis_result = expression_analysis_result.getHaving(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); std::optional actions; if (having_analysis_result.filter_actions) { - actions = std::move(*ActionsDAG::clone(&having_analysis_result.filter_actions->dag)); + actions = std::move(having_analysis_result.filter_actions->dag); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -886,7 +886,7 @@ bool addPreliminaryLimitOptimizationStepIfNeeded(QueryPlan & query_plan, * WINDOW functions. */ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, - const PlannerExpressionsAnalysisResult & expressions_analysis_result, + PlannerExpressionsAnalysisResult & expressions_analysis_result, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, @@ -922,7 +922,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, if (expressions_analysis_result.hasLimitBy()) { - const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); + auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1549,7 +1549,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasAggregation()) { - const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); + auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); if (aggregation_analysis_result.before_aggregation_actions) addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets); @@ -1568,7 +1568,7 @@ void Planner::buildPlanForQueryNode() * window functions, we can't execute ORDER BY and DISTINCT * now, on shard (first_stage). */ - const auto & window_analysis_result = expression_analysis_result.getWindow(); + auto & window_analysis_result = expression_analysis_result.getWindow(); if (window_analysis_result.before_window_actions) addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets); } @@ -1578,7 +1578,7 @@ void Planner::buildPlanForQueryNode() * Projection expressions, preliminary DISTINCT and before ORDER BY expressions * now, on shards (first_stage). */ - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) @@ -1594,7 +1594,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { - const auto & sort_analysis_result = expression_analysis_result.getSort(); + auto & sort_analysis_result = expression_analysis_result.getSort(); addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } @@ -1648,7 +1648,7 @@ void Planner::buildPlanForQueryNode() { if (expression_analysis_result.hasWindow()) { - const auto & window_analysis_result = expression_analysis_result.getWindow(); + auto & window_analysis_result = expression_analysis_result.getWindow(); if (expression_analysis_result.hasAggregation()) addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets); @@ -1658,7 +1658,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasQualify()) addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets); - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) @@ -1674,7 +1674,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { - const auto & sort_analysis_result = expression_analysis_result.getSort(); + auto & sort_analysis_result = expression_analysis_result.getSort(); addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } @@ -1727,7 +1727,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy()) { - const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); + auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1759,7 +1759,7 @@ void Planner::buildPlanForQueryNode() /// Project names is not done on shards, because initiator will not find columns in blocks if (!query_processing_info.isToAggregationState()) { - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets); } diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 820df7131a7..283fcac7aba 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -64,7 +64,7 @@ public: : projection_analysis_result(std::move(projection_analysis_result_)) {} - const ProjectionAnalysisResult & getProjection() const + ProjectionAnalysisResult & getProjection() { return projection_analysis_result; } @@ -74,7 +74,7 @@ public: return where_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getWhere() const + FilterAnalysisResult & getWhere() { return where_analysis_result; } @@ -89,7 +89,7 @@ public: return !aggregation_analysis_result.aggregation_keys.empty() || !aggregation_analysis_result.aggregate_descriptions.empty(); } - const AggregationAnalysisResult & getAggregation() const + AggregationAnalysisResult & getAggregation() { return aggregation_analysis_result; } @@ -104,7 +104,7 @@ public: return having_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getHaving() const + FilterAnalysisResult & getHaving() { return having_analysis_result; } @@ -119,7 +119,7 @@ public: return !window_analysis_result.window_descriptions.empty(); } - const WindowAnalysisResult & getWindow() const + WindowAnalysisResult & getWindow() { return window_analysis_result; } @@ -134,7 +134,7 @@ public: return qualify_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getQualify() const + FilterAnalysisResult & getQualify() { return qualify_analysis_result; } @@ -149,7 +149,7 @@ public: return sort_analysis_result.before_order_by_actions != nullptr; } - const SortAnalysisResult & getSort() const + SortAnalysisResult & getSort() { return sort_analysis_result; } @@ -164,7 +164,7 @@ public: return limit_by_analysis_result.before_limit_by_actions != nullptr; } - const LimitByAnalysisResult & getLimitBy() const + LimitByAnalysisResult & getLimitBy() { return limit_by_analysis_result; } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 4cf1b138bed..5acff9dac82 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -591,10 +591,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = std::move(*ActionsDAG::clone(&left_join_actions)); + result.left_join_expressions_actions = left_join_actions.clone(); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = std::move(*ActionsDAG::clone(&right_join_actions)); + result.right_join_expressions_actions = right_join_actions.clone(); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 94098f443d9..6f88c4527a4 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 5f15c5defac..0c6b71387b7 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 8666912514e..37e61a6c388 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -10,18 +10,18 @@ namespace DB::QueryPlanOptimizations { /// build actions DAG from stack of steps -static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) +static std::optional buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) - return nullptr; + return {}; - ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); + ActionsDAG path_actions = dag_stack.back()->clone(); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); + ActionsDAG clone = dag_stack.back()->clone(); dag_stack.pop_back(); - path_actions->mergeInplace(std::move(*clone)); + path_actions.mergeInplace(std::move(clone)); } return path_actions; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 411b20b1a32..73314f005b6 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -599,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - std::move(*ActionsDAG::clone(&filter->getExpression())), + filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -613,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 53f59198d0f..c48551732c9 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - std::move(*ActionsDAG::clone(&expression->getExpression()))); + expression->getExpression().clone()); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 71a7ca327b1..63b4e019066 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->prewhere_actions->clone()), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->row_level_filter->clone()), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(ActionsDAG::clone(&filter_step->getExpression()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(std::make_unique(filter_step->getExpression().clone()), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 99aaef6d054..252420e19fe 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -171,17 +171,17 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt } } -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAG & expression) +static void appendExpression(std::optional & dag, const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); + dag->mergeInplace(expression.clone()); else - dag = ActionsDAG::clone(&expression); + dag = expression.clone(); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, build a set of fixed columns. -void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns, size_t & limit) +void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, FixedColumns & fixed_columns, size_t & limit) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -330,7 +330,7 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) InputOrderInfoPtr buildInputOrderInfo( const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, const KeyDescription & sorting_key, size_t limit) @@ -507,7 +507,7 @@ struct AggregationInputOrder AggregationInputOrder buildInputOrderInfo( const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys, const ActionsDAG & sorting_key_dag, const Names & sorting_key_columns) @@ -693,7 +693,7 @@ AggregationInputOrder buildInputOrderInfo( InputOrderInfoPtr buildInputOrderInfo( const ReadFromMergeTree * reading, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, size_t limit) { @@ -709,7 +709,7 @@ InputOrderInfoPtr buildInputOrderInfo( InputOrderInfoPtr buildInputOrderInfo( ReadFromMerge * merge, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, size_t limit) { @@ -745,7 +745,7 @@ InputOrderInfoPtr buildInputOrderInfo( AggregationInputOrder buildInputOrderInfo( ReadFromMergeTree * reading, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys) { const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); @@ -760,7 +760,7 @@ AggregationInputOrder buildInputOrderInfo( AggregationInputOrder buildInputOrderInfo( ReadFromMerge * merge, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys) { const auto & tables = merge->getSelectedTables(); @@ -801,7 +801,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n const auto & description = sorting.getSortDescription(); size_t limit = sorting.getLimit(); - ActionsDAGPtr dag; + std::optional dag; FixedColumns fixed_columns; buildSortingDAG(node, dag, fixed_columns, limit); @@ -855,7 +855,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl const auto & keys = aggregating.getParams().keys; size_t limit = 0; - ActionsDAGPtr dag; + std::optional dag; FixedColumns fixed_columns; buildSortingDAG(node, dag, fixed_columns, limit); @@ -1076,13 +1076,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 4448d4b7869..ad89cec5f79 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -43,7 +43,7 @@ static DAGIndex buildDAGIndex(const ActionsDAG & dag) /// Required analysis info from aggregate projection. struct AggregateProjectionInfo { - ActionsDAGPtr before_aggregation; + std::optional before_aggregation; Names keys; AggregateDescriptions aggregates; @@ -78,7 +78,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = ActionsDAG::clone(&analysis_result.before_aggregation->dag); + info.before_aggregation = analysis_result.before_aggregation->dag.clone(); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index ad76976becc..571d1dd0cc1 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -68,9 +68,9 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea void QueryDAG::appendExpression(const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); + dag->mergeInplace(expression.clone()); else - dag = std::move(*ActionsDAG::clone(&expression)); + dag = expression.clone(); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -239,7 +239,8 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = ActionsDAG::clone(dag); + if (dag) + projection_query_info.filter_actions_dag = std::make_unique(dag->clone()); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index d0acd8221d4..7664822cc7e 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -43,10 +43,10 @@ namespace } } - void logActionsDAG(const String & prefix, const ActionsDAGPtr & actions) + void logActionsDAG(const String & prefix, const ActionsDAG & actions) { if constexpr (debug_logging_enabled) - LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG()); + LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions.dumpDAG()); } using DistinctColumns = std::set; @@ -65,19 +65,19 @@ namespace } /// build actions DAG from stack of steps - ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) + std::optional buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) - return nullptr; + return {}; - ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); + ActionsDAG path_actions = dag_stack.back()->clone(); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); + ActionsDAG clone = dag_stack.back()->clone(); logActionsDAG("DAG to merge", clone); dag_stack.pop_back(); - path_actions->mergeInplace(std::move(*clone)); + path_actions.mergeInplace(std::move(clone)); } return path_actions; } @@ -260,15 +260,15 @@ namespace if (distinct_columns.size() != inner_distinct_columns.size()) return false; - ActionsDAGPtr path_actions; + ActionsDAG path_actions; if (!dag_stack.empty()) { /// build actions DAG to find original column names - path_actions = buildActionsForPlanPath(dag_stack); + path_actions = std::move(*buildActionsForPlanPath(dag_stack)); logActionsDAG("distinct pass: merged DAG", path_actions); /// compare columns of two DISTINCTs - FindOriginalNodeForOutputName original_node_finder(*path_actions); + FindOriginalNodeForOutputName original_node_finder(path_actions); for (const auto & column : distinct_columns) { const auto * alias_node = original_node_finder.find(String(column)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index add53f9d6b3..5dda4ddc18b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -802,7 +802,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); + auto sorting_expr = metadata_for_reading->getSortingKey().expression; SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -1215,7 +1215,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); + auto sorting_expr = metadata_for_reading->getSortingKey().expression; if (prewhere_info) { @@ -1523,7 +1523,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*ActionsDAG::clone(&*filter_actions_dag))); + query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); buildIndexes( indexes, @@ -2004,7 +2004,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(std::move(*ActionsDAG::clone(result.sampling.filter_expression.get()))); + auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2137,7 +2137,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -2146,7 +2146,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -2172,7 +2172,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2182,7 +2182,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index 55c9b5e442e..b91debc8239 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 70457918de1..2554053064f 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -101,13 +101,16 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); - for (const auto & action : expression->getActions()) + if (actions_dag) { - settings.out << prefix << (first ? "Actions: " - : " "); - first = false; - settings.out << action.toString() << '\n'; + auto expression = std::make_shared(actions_dag->clone()); + for (const auto & action : expression->getActions()) + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } } } @@ -118,8 +121,11 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); - map.add("Expression", expression->toTree()); + if (actions_dag) + { + auto expression = std::make_shared(actions_dag->clone()); + map.add("Expression", expression->toTree()); + } } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 36ffc515f43..9601f821cc8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(std::move(*ActionsDAG::clone(&interpolate_description->actions))); + interpolate_actions = std::make_shared(interpolate_description->actions.clone()); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index ca31ffc9de5..c0875ed184d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,7 +265,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = std::move(*ActionsDAG::clone(filter_dag)); + auto filter_actions_dag = filter_dag->clone(); const auto * filter_actions_dag_node = filter_actions_dag.getOutputs().at(0); std::unordered_map node_to_result_node; @@ -319,7 +319,7 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node return index_hint.getActions().getOutputs(); /// Import the DAG and map argument pointers. - auto actions_clone = std::move(*ActionsDAG::clone(&index_hint.getActions())); + auto actions_clone = index_hint.getActions().clone(); chassert(storage); result_dag_or_null->mergeNodes(std::move(actions_clone), storage); return *storage; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 22289187cfa..f1df9e231c4 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)), actions_settings), + .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)), actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 589698fcc30..60f103fdb70 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = std::move(*ActionsDAG::clone(&*row_level_filter)); + prewhere_info->row_level_filter = row_level_filter->clone(); if (prewhere_actions) - prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(&*prewhere_actions)); + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c096504170e..aee4e4683ad 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -313,7 +313,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*ActionsDAG::clone(&actions_dag)), + actions_dag.clone(), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -322,7 +322,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*ActionsDAG::clone(&actions_dag)), + actions_dag.clone(), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -429,21 +429,23 @@ void StorageBuffer::read( if (query_info.prewhere_info->row_level_filter) { + auto actions = std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->row_level_filter)), actions_settings), + actions, query_info.prewhere_info->row_level_column_name, false); }); } + auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->prewhere_actions)), actions_settings), + actions, query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e16e2a07685..0e1568c8e79 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -663,7 +663,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&filter_info.actions)), + filter_info.actions.clone(), filter_info.column_name, filter_info.do_remove_column); @@ -1241,7 +1241,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag)), + filter_actions = std::make_shared(actions_dag.clone(), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1279,12 +1279,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(ActionsDAG::clone(&actions_dag), filter_column_name); + step->addFilter(std::make_unique(actions_dag.clone()), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&actions_dag)), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag.clone(), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 32c6a558340..146a54eda78 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -323,7 +323,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = std::move(*ActionsDAG::clone(&index_hint->getActions())); + auto index_hint_dag = index_hint->getActions().clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag.getOutputs()) if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) From 392183832834bf7041a0cae4fd27fc1112f51bc2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jul 2024 16:45:26 +0000 Subject: [PATCH 035/321] Fix runtime error while converting [Null] field to Array(Variant) --- src/Interpreters/convertFieldToType.cpp | 10 +++++----- .../03203_variant_convert_field_to_type_bug.reference | 0 .../03203_variant_convert_field_to_type_bug.sql | 5 +++++ 3 files changed, 10 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.reference create mode 100644 tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 184c263dbdb..9ee214f4415 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -356,7 +356,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID for (size_t i = 0; i < src_arr_size; ++i) { res[i] = convertFieldToType(src_arr[i], element_type); - if (res[i].isNull() && !element_type.isNullable()) + if (res[i].isNull() && !canContainNull(element_type)) { // See the comment for Tuples below. have_unconvertible_element = true; @@ -384,7 +384,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { const auto & element_type = *(type_tuple->getElements()[i]); res[i] = convertFieldToType(src_tuple[i], element_type); - if (!res[i].isNull() || element_type.isNullable()) + if (!res[i].isNull() || canContainNull(element_type)) continue; /* @@ -433,11 +433,11 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID updated_entry[0] = convertFieldToType(key, key_type); - if (updated_entry[0].isNull() && !key_type.isNullable()) + if (updated_entry[0].isNull() && !canContainNull(key_type)) have_unconvertible_element = true; updated_entry[1] = convertFieldToType(value, value_type); - if (updated_entry[1].isNull() && !value_type.isNullable()) + if (updated_entry[1].isNull() && !canContainNull(value_type)) have_unconvertible_element = true; res[i] = updated_entry; @@ -592,7 +592,7 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) { bool is_null = from_value.isNull(); - if (is_null && !to_type.isNullable() && !to_type.isLowCardinalityNullable()) + if (is_null && !canContainNull(to_type)) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); Field converted = convertFieldToType(from_value, to_type, from_type_hint); diff --git a/tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.reference b/tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.sql b/tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.sql new file mode 100644 index 00000000000..b73bb8ffa6d --- /dev/null +++ b/tests/queries/0_stateless/03203_variant_convert_field_to_type_bug.sql @@ -0,0 +1,5 @@ +set allow_experimental_variant_type=1; +set use_variant_as_common_type=1; + +SELECT * FROM numbers([tuple(1, 2), NULL], 2); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} + From 225af356c7ca3fd3c401f0fa8273d3dd751297fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 17:06:03 +0000 Subject: [PATCH 036/321] Better. --- src/Interpreters/ActionsDAG.cpp | 1 - src/Interpreters/ActionsDAG.h | 1 - src/Interpreters/ActionsVisitor.h | 8 +------- src/Interpreters/ExpressionAnalyzer.h | 3 --- src/Interpreters/WindowDescription.h | 1 - src/Interpreters/addMissingDefaults.h | 6 ------ 6 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 53e04f24829..85b2b38da17 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3150,7 +3150,6 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp } FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_) - //: actions(actions_) { const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 6f6c3f9bccb..76cc9327530 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -501,7 +501,6 @@ public: const ActionsDAG::Node * find(const String & name); private: - //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 496d9b9b587..5b638fc14c8 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -18,12 +18,6 @@ namespace DB class ASTExpressionList; class ASTFunction; -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; - -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; @@ -32,7 +26,7 @@ FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets); /** For ActionsVisitor - * A stack of ExpressionActions corresponding to nested lambda expressions. + * A stack of ActionsDAG corresponding to nested lambda expressions. * The new action should be added to the highest possible level. * For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)" * calculation of the product must be done outside the lambda expression (it does not depend on x), diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 737d36eb504..0c00247df85 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -38,9 +38,6 @@ using StorageMetadataPtr = std::shared_ptr; class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false); diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index 17bfe619c30..d51d9ca94d8 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -14,7 +14,6 @@ namespace DB class ASTFunction; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; struct WindowFunctionDescription { diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 5299bae9745..551583a0006 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -2,11 +2,6 @@ #include -#include -#include -#include - - namespace DB { @@ -15,7 +10,6 @@ class NamesAndTypesList; class ColumnsDescription; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; /** Adds three types of columns into block * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) From 4ebb189691c2d553887e9d49b52f9e0a45eaf004 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 17:59:59 +0000 Subject: [PATCH 037/321] Better. --- src/Planner/CollectTableExpressionData.cpp | 8 ++++---- src/Planner/PlannerJoinTree.cpp | 12 ++++++------ src/Planner/TableExpressionData.h | 6 +++--- src/Storages/SelectQueryInfo.cpp | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 1d85476636c..2fe62aa9be0 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -88,15 +88,15 @@ public: auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); - ActionsDAGPtr alias_column_actions_dag = std::make_unique(); + ActionsDAG alias_column_actions_dag; PlannerActionsVisitor actions_visitor(planner_context, false); - auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression()); + auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression()); if (outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size()); const auto & column_name = column_node->getColumnName(); - const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name); - alias_column_actions_dag->addOrReplaceInOutputs(alias_node); + const auto & alias_node = alias_column_actions_dag.addAlias(*outputs[0], column_name); + alias_column_actions_dag.addOrReplaceInOutputs(alias_node); table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 10b5a761d58..048bfa4b577 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,16 +591,16 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) } std::unique_ptr createComputeAliasColumnsStep( - const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) + std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs(); - for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) + for (auto & [column_name, alias_column_actions_dag] : alias_column_expressions) { - const auto & current_outputs = alias_column_actions_dag->getOutputs(); + const auto & current_outputs = alias_column_actions_dag.getOutputs(); action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); - merged_alias_columns_actions_dag.mergeNodes(std::move(*alias_column_actions_dag)); + merged_alias_columns_actions_dag.mergeNodes(std::move(alias_column_actions_dag)); } for (const auto * output_node : action_dag_outputs) @@ -996,7 +996,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } } - const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) { auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream()); @@ -1085,7 +1085,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan = std::move(subquery_planner).extractQueryPlan(); } - const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) { auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream()); diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 1d04fac3dc3..72412a869e4 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -73,7 +73,7 @@ public: } /// Add alias column - void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true) + void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAG actions_dag, bool is_selected_column = true) { alias_column_expressions.emplace(column.name, std::move(actions_dag)); addColumnImpl(column, column_identifier, is_selected_column); @@ -94,7 +94,7 @@ public: } /// Get ALIAS columns names mapped to expressions - const std::unordered_map & getAliasColumnExpressions() const + std::unordered_map & getAliasColumnExpressions() { return alias_column_expressions; } @@ -277,7 +277,7 @@ private: NameSet selected_column_names_set; /// Expression to calculate ALIAS columns - std::unordered_map alias_column_expressions; + std::unordered_map alias_column_expressions; /// Valid for table, table function, array join, query, union nodes ColumnNameToColumn column_name_to_column; diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp index d59ccf0dfaf..c9c96ed5837 100644 --- a/src/Storages/SelectQueryInfo.cpp +++ b/src/Storages/SelectQueryInfo.cpp @@ -18,7 +18,7 @@ std::unordered_map SelectQueryInfo::buildNod std::unordered_map node_name_to_input_node_column; if (planner_context) { - const auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) { From 4ef9cb6d7aa32aeb56c26bfa6ecad94beacba540 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 23:13:32 +0200 Subject: [PATCH 038/321] Fix style --- src/IO/ReadBufferFromFileBase.cpp | 5 ----- src/IO/ReadWriteBufferFromHTTP.cpp | 1 - 2 files changed, 6 deletions(-) diff --git a/src/IO/ReadBufferFromFileBase.cpp b/src/IO/ReadBufferFromFileBase.cpp index d42b12ba49b..b7a1438cff8 100644 --- a/src/IO/ReadBufferFromFileBase.cpp +++ b/src/IO/ReadBufferFromFileBase.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_FILE_SIZE; -} - ReadBufferFromFileBase::ReadBufferFromFileBase() : BufferWithOwnMemory(0) { } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 2a62b11aa44..4d27a78c8dc 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -72,7 +72,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int UNKNOWN_FILE_SIZE; } std::unique_ptr ReadWriteBufferFromHTTP::CallResult::transformToReadBuffer(size_t buf_size) && From 0bf9346b07dc6fb07180a4221477512ba4eae024 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 00:08:36 +0200 Subject: [PATCH 039/321] Update 03206_no_exceptions_clickhouse_local.sh --- .../queries/0_stateless/03206_no_exceptions_clickhouse_local.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh index 86839a228dc..00efd1f4591 100755 --- a/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh +++ b/tests/queries/0_stateless/03206_no_exceptions_clickhouse_local.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-fasttest -# Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so the grpc library is not built CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 7d9b7cc79611751adc6d22aa47c5e179228a2840 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 02:54:29 +0200 Subject: [PATCH 040/321] Introduce ASTDataType --- src/DataTypes/DataTypeFactory.cpp | 14 ++--- src/Databases/DatabasesCommon.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 15 +++-- .../InterpreterShowCreateQuery.cpp | 2 - .../MySQL/InterpretersMySQLDDLQuery.cpp | 28 ++++----- .../formatWithPossiblyHidingSecrets.h | 8 ++- src/Parsers/ASTColumnDeclaration.cpp | 4 -- src/Parsers/ASTDataType.cpp | 57 +++++++++++++++++++ src/Parsers/ASTDataType.h | 36 ++++++++++++ src/Parsers/ASTFunction.cpp | 4 -- src/Parsers/ASTFunction.h | 2 +- src/Parsers/IAST.h | 15 +++-- src/Parsers/ParserCreateQuery.cpp | 7 ++- src/Parsers/ParserCreateQuery.h | 8 +-- src/Parsers/ParserDataType.cpp | 14 ++--- .../StorageMaterializedPostgreSQL.cpp | 13 ++--- src/Storages/WindowView/StorageWindowView.cpp | 3 +- 17 files changed, 156 insertions(+), 76 deletions(-) create mode 100644 src/Parsers/ASTDataType.cpp create mode 100644 src/Parsers/ASTDataType.h diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index af37cde2846..45552e506cd 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -83,15 +83,9 @@ DataTypePtr DataTypeFactory::tryGet(const ASTPtr & ast) const template DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const { - if (const auto * func = ast->as()) + if (const auto * type = ast->as()) { - if (func->parameters) - { - if constexpr (nullptr_on_error) - return nullptr; - throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Data type cannot have multiple parenthesized parameters."); - } - return getImpl(func->name, func->arguments); + return getImpl(type->name, type->arguments); } if (const auto * ident = ast->as()) @@ -107,7 +101,7 @@ DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const if constexpr (nullptr_on_error) return nullptr; - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type."); + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type: {}.", ast->getID()); } DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fe0baf30e57..cacba581745 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -149,7 +149,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_ columns = metadata_ptr->columns.getAll(); for (const auto & column_name_and_type: columns) { - const auto & ast_column_declaration = std::make_shared(); + const auto ast_column_declaration = std::make_shared(); ast_column_declaration->name = column_name_and_type.name; /// parser typename { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index a846e23cd4f..032fc33ea16 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -12,9 +12,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -25,6 +25,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -432,7 +433,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co auto metadata_snapshot = storage->getInMemoryMetadataPtr(); for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) { - const auto & column_declaration = std::make_shared(); + const auto column_declaration = std::make_shared(); column_declaration->name = column_type_and_name.name; column_declaration->type = getColumnDeclaration(column_type_and_name.type); columns_expression_list->children.emplace_back(column_declaration); @@ -470,17 +471,15 @@ ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) c WhichDataType which(data_type); if (which.isNullable()) - return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isArray()) - return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isDateTime64()) - { - return makeASTFunction("DateTime64", std::make_shared(static_cast(6))); - } + return makeASTDataType("DateTime64", std::make_shared(static_cast(6))); - return std::make_shared(data_type->getName()); + return makeASTDataType(data_type->getName()); } void registerDatabasePostgreSQL(DatabaseFactory & factory) diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 0fca7b64d5a..ca5b7a3b5c1 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -1,9 +1,7 @@ #include #include -#include #include #include -#include #include #include #include diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4821d607d0e..f73965cfcc8 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ #include #include + namespace DB { @@ -95,22 +97,22 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) } ASTPtr data_type = declare_column->data_type; - auto * data_type_function = data_type->as(); + auto * data_type_node = data_type->as(); - if (data_type_function) + if (data_type_node) { - String type_name_upper = Poco::toUpper(data_type_function->name); + String type_name_upper = Poco::toUpper(data_type_node->name); if (is_unsigned) { /// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED) if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED") && !endsWith(type_name_upper, "UNSIGNED")) - data_type_function->name = type_name_upper + " UNSIGNED"; + data_type_node->name = type_name_upper + " UNSIGNED"; } if (type_name_upper == "SET") - data_type_function->arguments.reset(); + data_type_node->arguments.reset(); /// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs /// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3) @@ -119,7 +121,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) if (type_name_upper.find("ENUM") != String::npos) { UInt16 i = 0; - for (ASTPtr & child : data_type_function->arguments->children) + for (ASTPtr & child : data_type_node->arguments->children) { auto new_child = std::make_shared(); new_child->name = "equals"; @@ -133,10 +135,10 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) } if (type_name_upper == "DATE") - data_type_function->name = "Date32"; + data_type_node->name = "Date32"; } if (is_nullable) - data_type = makeASTFunction("Nullable", data_type); + data_type = makeASTDataType("Nullable", data_type); columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type)); } @@ -175,7 +177,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col return columns_description; } -static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, const NamesAndTypesList & columns) +static NamesAndTypesList getNames(const ASTDataType & expr, ContextPtr context, const NamesAndTypesList & columns) { if (expr.arguments->children.empty()) return NamesAndTypesList{}; @@ -219,9 +221,9 @@ static std::tuplechildren.empty()) { @@ -482,7 +484,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( { auto column_declaration = std::make_shared(); column_declaration->name = name; - column_declaration->type = makeASTFunction(type); + column_declaration->type = makeASTDataType(type); column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_expression = std::make_shared(default_value); column_declaration->children.emplace_back(column_declaration->type); diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 039bcbc2bca..ea8c295b169 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -1,11 +1,14 @@ #pragma once -#include "Access/ContextAccess.h" -#include "Interpreters/Context.h" + +#include +#include + #include namespace DB { + struct SecretHidingFormatSettings { // We can't store const Context& as there's a dangerous usage {.ctx = *getContext()} @@ -24,4 +27,5 @@ inline String format(const SecretHidingFormatSettings & settings) return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets); } + } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 4a8a3d2967d..c96499095d5 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include namespace DB @@ -15,8 +13,6 @@ ASTPtr ASTColumnDeclaration::clone() const if (type) { - // Type may be an ASTFunction (e.g. `create table t (a Decimal(9,0))`), - // so we have to clone it properly as well. res->type = type->clone(); res->children.push_back(res->type); } diff --git a/src/Parsers/ASTDataType.cpp b/src/Parsers/ASTDataType.cpp new file mode 100644 index 00000000000..3c17ae8c380 --- /dev/null +++ b/src/Parsers/ASTDataType.cpp @@ -0,0 +1,57 @@ +#include +#include +#include + + +namespace DB +{ + +String ASTDataType::getID(char delim) const +{ + return "DataType" + (delim + name); +} + +ASTPtr ASTDataType::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (arguments) + { + res->arguments = arguments->clone(); + res->children.push_back(res->arguments); + } + + return res; +} + +void ASTDataType::updateTreeHashImpl(SipHash & hash_state, bool) const +{ + hash_state.update(name.size()); + hash_state.update(name); + /// Children are hashed automatically. +} + +void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_function : "") << name; + + if (arguments && !arguments->children.empty()) + { + settings.ostr << '(' << (settings.hilite ? hilite_none : ""); + + for (size_t i = 0, size = arguments->children.size(); i < size; ++i) + { + if (i != 0) + settings.ostr << ", "; + + arguments->children[i]->formatImpl(settings, state, frame); + } + + settings.ostr << (settings.hilite ? hilite_function : "") << ')'; + } + + settings.ostr << (settings.hilite ? hilite_none : ""); +} + +} diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h new file mode 100644 index 00000000000..c8f3c6e2e9d --- /dev/null +++ b/src/Parsers/ASTDataType.h @@ -0,0 +1,36 @@ +#pragma once + +#include + + +namespace DB +{ + +/// AST for data types, e.g. UInt8 or Tuple(x UInt8, y Enum(a = 1)) +class ASTDataType : public IAST +{ +public: + String name; + ASTPtr arguments; + + String getID(char delim) const override; + ASTPtr clone() const override; + void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +template +std::shared_ptr makeASTDataType(const String & name, Args &&... args) +{ + auto function = std::make_shared(); + + function->name = name; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + + function->arguments->children = { std::forward(args)... }; + + return function; +} + +} diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f39229d7566..7d42b6d1e9c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -19,9 +18,6 @@ #include #include #include -#include - -#include using namespace std::literals; diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 3a94691f25d..be2b6beae54 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -46,7 +46,7 @@ public: NullsAction nulls_action = NullsAction::EMPTY; - /// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names. + /// do not print empty parentheses if there are no args - compatibility with engine names. bool no_empty_args = false; /// Specifies where this function-like expression is used. diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index ee70fed0f07..e7e2ff2ec4a 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -271,16 +271,15 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID()); } - // Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. - // You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. + /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. + /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const; - /* - * formatForLogging and formatForErrorMessage always hide secrets. This inconsistent - * behaviour is due to the fact such functions are called from Client which knows nothing about - * access rights and settings. Moreover, the only use case for displaying secrets are backups, - * and backup tools use only direct input and ignore logs and error messages. - */ + /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent + * behaviour is due to the fact such functions are called from Client which knows nothing about + * access rights and settings. Moreover, the only use case for displaying secrets are backups, + * and backup tools use only direct input and ignore logs and error messages. + */ String formatForLogging(size_t max_length = 0) const { return formatWithPossiblyHidingSensitiveData(max_length, true, false); diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 014dc7bd3bf..fa232954cd6 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -75,9 +76,9 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!close.ignore(pos, expected)) return false; - auto func = std::make_shared(); + auto func = std::make_shared(); tryGetIdentifierNameInto(name, func->name); - // FIXME(ilezhankin): func->no_empty_args = true; ? + func->arguments = columns; func->children.push_back(columns); node = func; @@ -749,7 +750,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto * table_id = table->as(); - // Shortcut for ATTACH a previously detached table + /// A shortcut for ATTACH a previously detached table. bool short_attach = attach && !from_path; if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon)) { diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index bb37491a366..7bd1d1bf588 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -101,17 +101,15 @@ class IParserColumnDeclaration : public IParserBase { public: explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) - : require_type(require_type_) - , allow_null_modifiers(allow_null_modifiers_) - , check_keywords_after_name(check_keywords_after_name_) + : require_type(require_type_) + , allow_null_modifiers(allow_null_modifiers_) + , check_keywords_after_name(check_keywords_after_name_) { } void enableCheckTypeKeyword() { check_type_keyword = true; } protected: - using ASTDeclarePtr = std::shared_ptr; - const char * getName() const override{ return "column declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index ad33c7e4558..63800819899 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -198,13 +199,12 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - auto function_node = std::make_shared(); - function_node->name = type_name; - function_node->no_empty_args = true; + auto data_type_node = std::make_shared(); + data_type_node->name = type_name; if (pos->type != TokenType::OpeningRoundBracket) { - node = function_node; + node = data_type_node; return true; } ++pos; @@ -222,10 +222,10 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; ++pos; - function_node->arguments = expr_list_args; - function_node->children.push_back(function_node->arguments); + data_type_node->arguments = expr_list_args; + data_type_node->children.push_back(data_type_node->arguments); - node = function_node; + node = data_type_node; return true; } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a904b29e12f..f7dde509d4e 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -7,10 +7,8 @@ #include #include -#include #include -#include #include #include #include @@ -22,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -295,7 +294,7 @@ std::shared_ptr StorageMaterializedPostgreSQL::getMaterial auto column_declaration = std::make_shared(); column_declaration->name = std::move(name); - column_declaration->type = makeASTFunction(type); + column_declaration->type = makeASTDataType(type); column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_expression = std::make_shared(default_value); @@ -312,17 +311,17 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d WhichDataType which(data_type); if (which.isNullable()) - return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isArray()) - return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); /// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it if (which.isDecimal()) { auto make_decimal_expression = [&](std::string type_name) { - auto ast_expression = std::make_shared(); + auto ast_expression = std::make_shared(); ast_expression->name = type_name; ast_expression->arguments = std::make_shared(); @@ -354,7 +353,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d return ast_expression; } - return std::make_shared(data_type->getName()); + return makeASTDataType(data_type->getName()); } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e15da0074d5..65a30b18e96 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -805,7 +806,7 @@ ASTPtr StorageWindowView::getInnerTableCreateQuery(const ASTPtr & inner_query, c { auto column_window = std::make_shared(); column_window->name = window_id_name; - column_window->type = std::make_shared("UInt32"); + column_window->type = makeASTDataType("UInt32"); columns_list->children.push_back(column_window); } From 2860aa514d0d8837bd72a87390686b20e05ecae7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 03:50:38 +0200 Subject: [PATCH 041/321] Fix style --- src/DataTypes/DataTypeFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 45552e506cd..db6e1738d22 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNKNOWN_TYPE; - extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int UNEXPECTED_AST_STRUCTURE; extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } From 985f398925266c5867cd25fcdf655a5a306928fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 05:33:44 +0200 Subject: [PATCH 042/321] Fix error --- src/Client/ClientBase.cpp | 1 - src/DataTypes/DataTypeAggregateFunction.cpp | 4 +-- src/Parsers/ParserDataType.cpp | 36 ++++++++++++++------- src/TableFunctions/ITableFunction.cpp | 1 - 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13dce05cabc..95d2dff54e6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -70,7 +70,6 @@ #include #include -#include #include #include #include diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index ef7d86d2a81..09175617bf1 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -257,8 +257,8 @@ static DataTypePtr create(const ASTPtr & arguments) } else throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected AST element passed as aggregate function name for data type AggregateFunction. " - "Must be identifier or function."); + "Unexpected AST element {} passed as aggregate function name for data type AggregateFunction. " + "Must be identifier or function", data_type_ast->getID()); for (size_t i = argument_types_start_idx; i < arguments->children.size(); ++i) argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 63800819899..78b5aaa93a6 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -70,20 +70,32 @@ private: DynamicArgumentsParser parser; return parser.parse(pos, node, expected); } - - ParserNestedTable nested_parser; - ParserDataType data_type_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - - const char * operators[] = {"=", "equals", nullptr}; - ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); - - if (pos->type == TokenType::BareWord && std::string_view(pos->begin, pos->size()) == "Nested") + else if (type_name == "Nested") + { + ParserNestedTable nested_parser; return nested_parser.parse(pos, node, expected); + } + else if (type_name == "AggregateFunction") + { + ParserFunction function_parser; + ParserIdentifier identifier_parser; + ParserAllCollectionsOfLiterals literal_parser(false); + return literal_parser.parse(pos, node, expected) + || identifier_parser.parse(pos, node, expected) + || function_parser.parse(pos, node, expected); + } + else + { + ParserDataType data_type_parser; + ParserAllCollectionsOfLiterals literal_parser(false); - return enum_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || data_type_parser.parse(pos, node, expected); + const char * operators[] = {"=", "equals", nullptr}; + ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); + + return enum_parser.parse(pos, node, expected) + || literal_parser.parse(pos, node, expected) + || data_type_parser.parse(pos, node, expected); + } } std::string_view type_name; diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index e5676c5c25d..916ff7ec022 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include From 15be94ee14a7affe6643dd4c3ac1b104e3c69eeb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:47:27 +0200 Subject: [PATCH 043/321] Update src/Parsers/ASTDataType.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Parsers/ASTDataType.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h index c8f3c6e2e9d..abe5cbb8626 100644 --- a/src/Parsers/ASTDataType.h +++ b/src/Parsers/ASTDataType.h @@ -22,13 +22,13 @@ public: template std::shared_ptr makeASTDataType(const String & name, Args &&... args) { - auto function = std::make_shared(); + auto data_type = std::make_shared(); - function->name = name; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); + data_type->name = name; + data_type->arguments = std::make_shared(); + data_type->children.push_back(function->arguments); - function->arguments->children = { std::forward(args)... }; + data_type->arguments->children = { std::forward(args)... }; return function; } From 87fa2c64e96c6bac67275207bd708ac231fa9fb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:48:00 +0200 Subject: [PATCH 044/321] Apply review comments --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index f7dde509d4e..e795cd9c6c6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -345,7 +345,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d if (which.isDateTime64()) { - auto ast_expression = std::make_shared(); + auto ast_expression = std::make_shared(); ast_expression->name = "DateTime64"; ast_expression->arguments = std::make_shared(); From 88dce34be907863b6f3cf6890be87b0d7278a101 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:48:31 +0200 Subject: [PATCH 045/321] Fix error --- src/Parsers/ASTDataType.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h index abe5cbb8626..d9427c2fd9e 100644 --- a/src/Parsers/ASTDataType.h +++ b/src/Parsers/ASTDataType.h @@ -26,11 +26,11 @@ std::shared_ptr makeASTDataType(const String & name, Args &&... arg data_type->name = name; data_type->arguments = std::make_shared(); - data_type->children.push_back(function->arguments); + data_type->children.push_back(data_type->arguments); data_type->arguments->children = { std::forward(args)... }; - return function; + return data_type; } } From 4d4e0901881221b39cce0e0527f530fe90eb7ad3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:58:06 +0200 Subject: [PATCH 046/321] Fix build --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index e795cd9c6c6..e7b58841c4e 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include From 2832f8c57e37a1fc7d0c91b9ad621785c6d2a5a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 04:59:04 +0200 Subject: [PATCH 047/321] Fix a typo --- src/Databases/DatabasesCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index cacba581745..fdbdb610275 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -164,7 +164,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_ if (!parser.parse(pos, ast_type, expected)) { if (throw_on_error) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parser metadata of {}.{}", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parse metadata of {}.{}", backQuote(table_id.database_name), backQuote(table_id.table_name)); else return nullptr; From 6a7a4df1eca0cda4fd2efdcc8aaf2e8741f4cbcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:02:26 +0200 Subject: [PATCH 048/321] Fix error --- src/Parsers/ParserDataType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 78b5aaa93a6..af1a299a887 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -75,7 +75,7 @@ private: ParserNestedTable nested_parser; return nested_parser.parse(pos, node, expected); } - else if (type_name == "AggregateFunction") + else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") { ParserFunction function_parser; ParserIdentifier identifier_parser; From cc201745620490c7d885a9e45d9f4b92f7492c10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:19:58 +0200 Subject: [PATCH 049/321] Fix bad code: it was catching exceptions --- src/IO/WithFileSize.cpp | 48 +++++++++---------- ...ry_and_native_with_binary_encoded_types.sh | 4 +- 2 files changed, 25 insertions(+), 27 deletions(-) diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 3660d962c08..8cea12fa200 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -14,40 +14,38 @@ namespace ErrorCodes } template -static size_t getFileSize(T & in) +static std::optional tryGetFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - { return with_file_size->getFileSize(); - } + + return std::nullopt; +} + +template +static size_t getFileSize(T & in) +{ + if (auto maybe_size = tryGetFileSize(in)) + return *maybe_size; throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } -size_t getFileSizeFromReadBuffer(ReadBuffer & in) -{ - if (auto * delegate = dynamic_cast(&in)) - { - return getFileSize(delegate->getWrappedReadBuffer()); - } - else if (auto * compressed = dynamic_cast(&in)) - { - return getFileSize(compressed->getWrappedReadBuffer()); - } - - return getFileSize(in); -} - std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in) { - try - { - return getFileSizeFromReadBuffer(in); - } - catch (...) - { - return std::nullopt; - } + if (auto * delegate = dynamic_cast(&in)) + return tryGetFileSize(delegate->getWrappedReadBuffer()); + else if (auto * compressed = dynamic_cast(&in)) + return tryGetFileSize(compressed->getWrappedReadBuffer()); + return tryGetFileSize(in); +} + +size_t getFileSizeFromReadBuffer(ReadBuffer & in) +{ + if (auto maybe_size = tryGetFileSizeFromReadBuffer(in)) + return *maybe_size; + + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } bool isBufferWithFileSize(const ReadBuffer & in) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index 723b11ad620..0c585d36348 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test { - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" } test "materialize(42)::UInt8" From e1a24c9dd6f6320ce02714265e91e83f8dbf43f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:45:57 +0200 Subject: [PATCH 050/321] Fix error --- src/Parsers/ParserDataType.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index af1a299a887..c5da4a32e92 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -80,9 +79,9 @@ private: ParserFunction function_parser; ParserIdentifier identifier_parser; ParserAllCollectionsOfLiterals literal_parser(false); - return literal_parser.parse(pos, node, expected) - || identifier_parser.parse(pos, node, expected) - || function_parser.parse(pos, node, expected); + return function_parser.parse(pos, node, expected) + || literal_parser.parse(pos, node, expected) + || identifier_parser.parse(pos, node, expected); } else { From bc1146389617f9e9198d0dd25eea89a9206421ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 03:16:23 +0200 Subject: [PATCH 051/321] Fix error --- src/Parsers/ParserDataType.cpp | 142 +++++++++++++++++++-------------- 1 file changed, 84 insertions(+), 58 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index c5da4a32e92..2edb0141e12 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -46,60 +46,6 @@ private: } }; -/// Wrapper to allow mixed lists of nested and normal types. -/// Parameters are either: -/// - Nested table elements; -/// - Enum element in form of 'a' = 1; -/// - literal; -/// - Dynamic type arguments; -/// - another data type (or identifier); -class ParserDataTypeArgument : public IParserBase -{ -public: - explicit ParserDataTypeArgument(std::string_view type_name_) : type_name(type_name_) - { - } - -private: - const char * getName() const override { return "data type argument"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override - { - if (type_name == "Dynamic") - { - DynamicArgumentsParser parser; - return parser.parse(pos, node, expected); - } - else if (type_name == "Nested") - { - ParserNestedTable nested_parser; - return nested_parser.parse(pos, node, expected); - } - else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") - { - ParserFunction function_parser; - ParserIdentifier identifier_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - return function_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || identifier_parser.parse(pos, node, expected); - } - else - { - ParserDataType data_type_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - - const char * operators[] = {"=", "equals", nullptr}; - ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); - - return enum_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || data_type_parser.parse(pos, node, expected); - } - } - - std::string_view type_name; -}; - } bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) @@ -221,11 +167,91 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; /// Parse optional parameters - ParserList args_parser(std::make_unique(type_name), std::make_unique(TokenType::Comma)); - ASTPtr expr_list_args; + ASTPtr expr_list_args = std::make_shared(); + + /// Allow mixed lists of nested and normal types. + /// Parameters are either: + /// - Nested table elements; + /// - Enum element in form of 'a' = 1; + /// - literal; + /// - Dynamic type arguments; + /// - another data type (or identifier); + + size_t arg_num = 0; + bool have_version_of_aggregate_function = false; + while (true) + { + if (arg_num > 0) + { + if (pos->type == TokenType::Comma) + ++pos; + else + break; + } + + ASTPtr arg; + if (type_name == "Dynamic") + { + DynamicArgumentsParser parser; + parser.parse(pos, arg, expected); + } + else if (type_name == "Nested") + { + ParserNestedTable nested_parser; + nested_parser.parse(pos, arg, expected); + } + else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") + { + /// This is less trivial. + /// The first optional argument for AggregateFunction is a numeric literal, defining the version. + /// The next argument is the function name, optionally with parameters. + /// Subsequent arguments are data types. + + if (arg_num == 0 && type_name == "AggregateFunction") + { + ParserUnsignedInteger version_parser; + if (version_parser.parse(pos, arg, expected)) + { + have_version_of_aggregate_function = true; + expr_list_args->children.emplace_back(std::move(arg)); + ++arg_num; + continue; + } + } + + if (arg_num == (have_version_of_aggregate_function ? 1 : 0)) + { + ParserFunction function_parser; + ParserIdentifier identifier_parser; + function_parser.parse(pos, arg, expected) + || identifier_parser.parse(pos, arg, expected); + } + else + { + ParserDataType data_type_parser; + data_type_parser.parse(pos, arg, expected); + } + } + else + { + ParserDataType data_type_parser; + ParserAllCollectionsOfLiterals literal_parser(false); + + const char * operators[] = {"=", "equals", nullptr}; + ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); + + enum_parser.parse(pos, arg, expected) + || literal_parser.parse(pos, arg, expected) + || data_type_parser.parse(pos, arg, expected); + } + + if (!arg) + break; + + expr_list_args->children.emplace_back(std::move(arg)); + ++arg_num; + } - if (!args_parser.parse(pos, expr_list_args, expected)) - return false; if (pos->type == TokenType::Comma) // ignore trailing comma inside Nested structures like Tuple(Int, Tuple(Int, String),) ++pos; From 38126bb436c3f46cfdc321abca421f7fae969f5b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 19 Jul 2024 19:09:55 +0200 Subject: [PATCH 052/321] 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 053/321] 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 a373b62bbf8083ffa96210fc1c959f13939526fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 00:04:52 +0200 Subject: [PATCH 054/321] Better diagnostics in functional tests --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0647ed02839..f43bb5da33d 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -264,7 +264,7 @@ if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. timeout_with_logging "$TIMEOUT" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \ - | sed 's/All tests have finished//' | sed 's/No tests were run//' ||: + | sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||: fi timeout_with_logging "$TIMEOUT" bash -c run_tests ||: From 134c0065407bd3f9394a720fbdfef7edf241ef84 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 00:08:25 +0200 Subject: [PATCH 055/321] Whitespace --- docker/test/stateless/utils.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index c3bb8ae9ea4..6b7b659296b 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -47,7 +47,7 @@ function timeout_with_logging() { if [[ "${exit_code}" -eq "124" ]] then - echo "The command 'timeout ${*}' has been killed by timeout" + echo "The command 'timeout ${*}' has been killed by timeout" fi return $exit_code From ba6b7b86ba3e868cd001efbce2c6cf8a5236a024 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 00:09:53 +0200 Subject: [PATCH 056/321] Log messages --- docker/test/stateless/utils.lib | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index 6b7b659296b..cb257536c36 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -45,9 +45,12 @@ function timeout_with_logging() { timeout -s TERM --preserve-status "${@}" || exit_code="${?}" + echo "Checking if it is a timeout. The code 124 will indicate a timeout." if [[ "${exit_code}" -eq "124" ]] then - echo "The command 'timeout ${*}' has been killed by timeout" + echo "The command 'timeout ${*}' has been killed by timeout." + else + echo "No, it isn't a timeout." fi return $exit_code From 5ae3a421e0bf90c3d1755371fe1d6ff5662207ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 00:12:02 +0200 Subject: [PATCH 057/321] Copy-paste --- docker/test/fasttest/run.sh | 5 ++++- docker/test/stateful/run.sh | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 970bf12a81a..26283afc86a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -261,9 +261,12 @@ function timeout_with_logging() { timeout -s TERM --preserve-status "${@}" || exit_code="${?}" + echo "Checking if it is a timeout. The code 124 will indicate a timeout." if [[ "${exit_code}" -eq "124" ]] then - echo "The command 'timeout ${*}' has been killed by timeout" + echo "The command 'timeout ${*}' has been killed by timeout." + else + echo "No, it isn't a timeout." fi return $exit_code diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 857385f4715..3a0e3a8be48 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -251,9 +251,12 @@ function timeout_with_logging() { timeout -s TERM --preserve-status "${@}" || exit_code="${?}" + echo "Checking if it is a timeout. The code 124 will indicate a timeout." if [[ "${exit_code}" -eq "124" ]] then - echo "The command 'timeout ${*}' has been killed by timeout" + echo "The command 'timeout ${*}' has been killed by timeout." + else + echo "No, it isn't a timeout." fi return $exit_code From 444303cb7117c92e578ad4ea20f7c0001edb3c8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 00:25:57 +0200 Subject: [PATCH 058/321] Better diagnostics --- docker/test/stateless/run.sh | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index f43bb5da33d..b24af431ff1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -247,12 +247,22 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" + TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800)) + START_TIME=${SECONDS} set +e - timeout -k 60m -s TERM --preserve-status 140m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s \ + clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e + DURATION=$((START_TIME - SECONDS)) + + echo "Elapsed ${DURATION} seconds." + if [[ $DURATION -ge $TIMEOUT ]] + then + echo "It looks like the command is terminated by the timeout, which is ${TIMEOUT} seconds." + fi } export -f run_tests From a3a4548d96dac4b480e3a54519cfa34fab17ce4c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 20 Jul 2024 15:01:29 +0200 Subject: [PATCH 059/321] 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 060/321] 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 8786d9b5dd5f93d5c1f22c4d618093d69f8d57a3 Mon Sep 17 00:00:00 2001 From: joelynch Date: Sun, 21 Jul 2024 20:00:38 +0200 Subject: [PATCH 061/321] Ensure COMMENT clause works for all table engines --- src/Databases/SQLite/DatabaseSQLite.cpp | 1 + src/Storages/Kafka/StorageKafka.cpp | 10 +++++++--- src/Storages/Kafka/StorageKafka.h | 1 + src/Storages/NATS/StorageNATS.cpp | 4 +++- src/Storages/NATS/StorageNATS.h | 1 + .../PostgreSQL/StorageMaterializedPostgreSQL.cpp | 1 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 +++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 1 + src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 1 + src/Storages/StorageExecutable.cpp | 7 ++++--- src/Storages/StorageExecutable.h | 3 ++- src/Storages/StorageKeeperMap.cpp | 1 + src/Storages/StorageSQLite.cpp | 4 +++- src/Storages/StorageSQLite.h | 1 + src/Storages/WindowView/StorageWindowView.cpp | 4 +++- src/Storages/WindowView/StorageWindowView.h | 1 + src/TableFunctions/TableFunctionExecutable.cpp | 9 ++++++++- src/TableFunctions/TableFunctionSQLite.cpp | 2 +- 18 files changed, 43 insertions(+), 13 deletions(-) diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 132a978140c..471730fce29 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -154,6 +154,7 @@ StoragePtr DatabaseSQLite::fetchTable(const String & table_name, ContextPtr loca table_name, ColumnsDescription{*columns}, ConstraintsDescription{}, + /* comment = */ "", local_context); return storage; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 809401bb279..3aad64a0cfb 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -418,8 +418,11 @@ namespace } StorageKafka::StorageKafka( - const StorageID & table_id_, ContextPtr context_, - const ColumnsDescription & columns_, std::unique_ptr kafka_settings_, + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & comment, + std::unique_ptr kafka_settings_, const String & collection_name_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -451,6 +454,7 @@ StorageKafka::StorageKafka( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); @@ -1317,7 +1321,7 @@ void registerStorageKafka(StorageFactory & factory) "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); } - return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); }; factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index fa4affbda36..31e1a6076b6 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -40,6 +40,7 @@ public: const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, + const String & comment, std::unique_ptr kafka_settings_, const String & collection_name_); diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 8f0e2d76473..9d728c3395f 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -49,6 +49,7 @@ StorageNATS::StorageNATS( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, + const String & comment, std::unique_ptr nats_settings_, LoadingStrictnessLevel mode) : IStorage(table_id_) @@ -87,6 +88,7 @@ StorageNATS::StorageNATS( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); setVirtuals(createVirtuals(nats_settings->nats_handle_error_mode)); @@ -760,7 +762,7 @@ void registerStorageNATS(StorageFactory & factory) if (!nats_settings->nats_subjects.changed) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `nats_subjects` setting"); - return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.mode); + return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(nats_settings), args.mode); }; factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 41d77acfde6..5fca8cb0163 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -23,6 +23,7 @@ public: const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, + const String & comment, std::unique_ptr nats_settings_, LoadingStrictnessLevel mode); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a904b29e12f..f4c38a52a3f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -571,6 +571,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); + metadata.setComment(args.comment); if (args.mode <= LoadingStrictnessLevel::CREATE && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f3d2aff68c8..9e3c40071b5 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -70,6 +70,7 @@ StorageRabbitMQ::StorageRabbitMQ( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, + const String & comment, std::unique_ptr rabbitmq_settings_, LoadingStrictnessLevel mode) : IStorage(table_id_) @@ -145,6 +146,7 @@ StorageRabbitMQ::StorageRabbitMQ( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode)); @@ -1288,7 +1290,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) if (!rabbitmq_settings->rabbitmq_format.changed) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting"); - return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(rabbitmq_settings), args.mode); + return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index b8fab5825e4..fed80a4357b 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -26,6 +26,7 @@ public: const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, + const String & comment, std::unique_ptr rabbitmq_settings_, LoadingStrictnessLevel mode); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 409703c84c6..fafc72da04e 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -691,6 +691,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); + metadata.setComment(args.comment); if (!args.storage_def->primary_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key"); diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 381c20c616d..0094723e3fd 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -77,7 +77,8 @@ StorageExecutable::StorageExecutable( const ExecutableSettings & settings_, const std::vector & input_queries_, const ColumnsDescription & columns, - const ConstraintsDescription & constraints) + const ConstraintsDescription & constraints, + const String & comment) : IStorage(table_id_) , settings(settings_) , input_queries(input_queries_) @@ -86,6 +87,7 @@ StorageExecutable::StorageExecutable( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); ShellCommandSourceCoordinator::Configuration configuration @@ -237,7 +239,7 @@ void registerStorageExecutable(StorageFactory & factory) settings.loadFromQuery(*args.storage_def); auto global_context = args.getContext()->getGlobalContext(); - return std::make_shared(args.table_id, format, settings, input_queries, columns, constraints); + return std::make_shared(args.table_id, format, settings, input_queries, columns, constraints, args.comment); }; StorageFactory::StorageFeatures storage_features; @@ -255,4 +257,3 @@ void registerStorageExecutable(StorageFactory & factory) } } - diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 2be2a84ab49..6748bb3223e 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -22,7 +22,8 @@ public: const ExecutableSettings & settings, const std::vector & input_queries, const ColumnsDescription & columns, - const ConstraintsDescription & constraints); + const ConstraintsDescription & constraints, + const String & comment); String getName() const override { diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 587cb621362..16caf01955e 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -1280,6 +1280,7 @@ StoragePtr create(const StorageFactory::Arguments & args) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); + metadata.setComment(args.comment); if (!args.storage_def->primary_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageKeeperMap requires one column in primary key"); diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 85417a2f2a4..b90b15f3b99 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -50,6 +50,7 @@ StorageSQLite::StorageSQLite( const String & remote_table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -71,6 +72,7 @@ StorageSQLite::StorageSQLite( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); + storage_metadata.setComment(comment); } @@ -211,7 +213,7 @@ void registerStorageSQLite(StorageFactory & factory) auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */ args.mode <= LoadingStrictnessLevel::CREATE); return std::make_shared(args.table_id, sqlite_db, database_path, - table_name, args.columns, args.constraints, args.getContext()); + table_name, args.columns, args.constraints, args.comment, args.getContext()); }, { .supports_schema_inference = true, diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index ed673123fe0..97638ac04cb 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -27,6 +27,7 @@ public: const String & remote_table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_); std::string getName() const override { return "SQLite"; } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e15da0074d5..7e1bca7d0d6 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1188,6 +1188,7 @@ StorageWindowView::StorageWindowView( ContextPtr context_, const ASTCreateQuery & query, const ColumnsDescription & columns_, + const String & comment, LoadingStrictnessLevel mode) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -1206,6 +1207,7 @@ StorageWindowView::StorageWindowView( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); /// If the target table is not set, use inner target table @@ -1761,7 +1763,7 @@ void registerStorageWindowView(StorageFactory & factory) "Experimental WINDOW VIEW feature " "is not enabled (the setting 'allow_experimental_window_view')"); - return std::make_shared(args.table_id, args.getLocalContext(), args.query, args.columns, args.mode); + return std::make_shared(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment, args.mode); }); } diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 14ac65091d3..38fca512ed9 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -111,6 +111,7 @@ public: ContextPtr context_, const ASTCreateQuery & query, const ColumnsDescription & columns_, + const String & comment, LoadingStrictnessLevel mode); String getName() const override { return "WindowView"; } diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 2c3802e8667..cccd3587bc7 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -170,7 +170,14 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, if (settings_query != nullptr) settings.applyChanges(settings_query->as()->changes); - auto storage = std::make_shared(storage_id, format, settings, input_queries, getActualTableStructure(context, is_insert_query), ConstraintsDescription{}); + auto storage = std::make_shared( + storage_id, + format, + settings, + input_queries, + getActualTableStructure(context, is_insert_query), + ConstraintsDescription{}, + /* comment = */ ""); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index e367e05bf73..87353025d1d 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -57,7 +57,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, sqlite_db, database_path, remote_table_name, - cached_columns, ConstraintsDescription{}, context); + cached_columns, ConstraintsDescription{}, /* comment = */ "", context); storage->startup(); return storage; From db549c93a18f49540676ae53bc04e75b85705ddb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:34:34 +0200 Subject: [PATCH 062/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 4d27a78c8dc..cea1a272401 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -123,7 +123,16 @@ void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, s std::optional ReadWriteBufferFromHTTP::tryGetFileSize() { if (!file_info) - file_info = getFileInfo(); + { + try + { + file_info = getFileInfo(); + } + catch (const HTTPException & e) + { + return std::nullopt; + } + } return file_info->file_size; } @@ -679,7 +688,7 @@ std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { file_info = getFileInfo(); } - catch (...) + catch (const HTTPException & e) { return std::nullopt; } @@ -700,7 +709,7 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() { getHeadResponse(response); } - catch (HTTPException & e) + catch (const HTTPException & e) { /// Maybe the web server doesn't support HEAD requests. /// E.g. webhdfs reports status 400. From 32f624eebaa560f4c9d6bf9145931270098e8db1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:35:10 +0200 Subject: [PATCH 063/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index cea1a272401..961e8dd6425 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -128,7 +128,7 @@ std::optional ReadWriteBufferFromHTTP::tryGetFileSize() { file_info = getFileInfo(); } - catch (const HTTPException & e) + catch (const HTTPException &) { return std::nullopt; } From dc601dc7455895574143f5baf345731d437bf8d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:37:15 +0200 Subject: [PATCH 064/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 961e8dd6425..85230957b3f 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -688,7 +688,7 @@ std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { file_info = getFileInfo(); } - catch (const HTTPException & e) + catch (const HTTPException &) { return std::nullopt; } From a564f70b66367ee4363d46e45eb5a9c66f131fec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:09:39 +0200 Subject: [PATCH 065/321] Fix error --- src/Parsers/ParserAlterQuery.cpp | 2 -- src/Parsers/ParserCreateIndexQuery.cpp | 4 ++-- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 28dbf781011..dbefb0cb966 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -9,8 +9,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 2fa34696c58..ddefb3d37fb 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -21,7 +21,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ParserToken close_p(TokenType::ClosingRoundBracket); ParserOrderByExpressionList order_list_p; - ParserDataType data_type_p; + ParserFunction type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -68,7 +68,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { - if (!data_type_p.parse(pos, type, expected)) + if (!type_p.parse(pos, type, expected)) return false; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index fa232954cd6..3dba58546af 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -179,7 +179,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_granularity(Keyword::GRANULARITY); ParserIdentifier name_p; - ParserDataType data_type_p; + ParserFunction type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -197,7 +197,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!s_type.ignore(pos, expected)) return false; - if (!data_type_p.parse(pos, type, expected)) + if (!type_p.parse(pos, type, expected)) return false; if (s_granularity.ignore(pos, expected)) From bbbf2fec88a331281bf51d3a02bd3f476e3bf6ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:37:54 +0200 Subject: [PATCH 066/321] Fix error --- src/Parsers/ParserCreateIndexQuery.cpp | 9 +++++++-- src/Parsers/ParserCreateQuery.cpp | 7 ++++++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index ddefb3d37fb..8a4c1c0b17a 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -7,9 +7,9 @@ #include #include #include -#include #include + namespace DB { @@ -69,7 +69,12 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { if (!type_p.parse(pos, type, expected)) - return false; + { + if (ParserIdentifier().parse(pos, type, expected)) + type = makeASTFunction(type->as().name()); + else + return false; + } } if (s_granularity.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 3dba58546af..bff5da4a536 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -198,7 +198,12 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!type_p.parse(pos, type, expected)) - return false; + { + if (name_p.parse(pos, type, expected)) + type = makeASTFunction(type->as().name()); + else + return false; + } if (s_granularity.ignore(pos, expected)) { From 885acd3aa80d421e82f75150b4152e227ca0fba4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 09:42:20 +0200 Subject: [PATCH 067/321] Compatibility --- src/Parsers/ParserCreateIndexQuery.cpp | 3 +++ src/Parsers/ParserCreateQuery.cpp | 3 +++ 2 files changed, 6 insertions(+) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 8a4c1c0b17a..2761c99738b 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -71,7 +71,10 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!type_p.parse(pos, type, expected)) { if (ParserIdentifier().parse(pos, type, expected)) + { type = makeASTFunction(type->as().name()); + type->as().no_empty_args = true; + } else return false; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index bff5da4a536..9aaecd84f59 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -200,7 +200,10 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!type_p.parse(pos, type, expected)) { if (name_p.parse(pos, type, expected)) + { type = makeASTFunction(type->as().name()); + type->as().no_empty_args = true; + } else return false; } From 8217dcccc1438ec6186e8db53e17429a7060183f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 09:53:56 +0200 Subject: [PATCH 068/321] 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 069/321] 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 41218ad01889cce72569a840461b54dbf9f4b832 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 22 Jul 2024 12:03:21 +0200 Subject: [PATCH 070/321] Stateless tests: add no-parallel tag for high-load tests --- .../01076_cache_dictionary_datarace_exception_ptr.sh | 2 +- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- .../0_stateless/01301_aggregate_state_exception_memory_leak.sh | 2 +- .../0_stateless/01302_aggregate_state_exception_memory_leak.sh | 2 +- tests/queries/0_stateless/02372_data_race_in_avro.sh | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh index dcd15718416..e003d2a26da 100755 --- a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh +++ b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race +# Tags: race, no-parallel # This is a monkey test used to trigger sanitizers. diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 2ab7f883367..8344bb6f426 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-ordinary-database, no-debug +# Tags: long, no-parallel, no-ordinary-database, no-debug # Test is too heavy, avoid parallel run in Flaky Check # shellcheck disable=SC2119 diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 47fe7a9c7d9..d74092d828d 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh index a521accb082..bbf2fd9177a 100755 --- a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.sh b/tests/queries/0_stateless/02372_data_race_in_avro.sh index 49c34e31923..50a7ae1e3c5 100755 --- a/tests/queries/0_stateless/02372_data_race_in_avro.sh +++ b/tests/queries/0_stateless/02372_data_race_in_avro.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 378502a331c60cc023e0c965611237cb5a3cfb47 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 22 Jul 2024 12:54:05 +0200 Subject: [PATCH 071/321] Stateless tests: sync tests with private --- .../01651_lc_insert_tiny_log.reference | 12 +- .../0_stateless/01651_lc_insert_tiny_log.sql | 6 +- .../01753_direct_dictionary_simple_key.sql | 2 +- .../0_stateless/02372_analyzer_join.reference | 688 +++++++++--------- .../0_stateless/02372_analyzer_join.sql.j2 | 44 +- .../02373_analyzer_join_use_nulls.reference | 16 +- .../02373_analyzer_join_use_nulls.sql | 16 +- .../02992_all_columns_should_have_comment.sql | 6 +- 8 files changed, 397 insertions(+), 393 deletions(-) diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference b/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference index 3da44c57b27..5cc8909b6c8 100644 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference @@ -1,12 +1,12 @@ 10000000 10000000 1274991808 -20000000 -20000000 2549983616 +30000000 +30000000 3824991808 10000000 10000000 1274991808 -20000000 -20000000 2549983616 +30000000 +30000000 3824991808 10000000 10000000 1274991808 -20000000 -20000000 2549983616 +30000000 +30000000 3824991808 diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql index d405bb01fd9..d11c9120c61 100644 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql @@ -8,7 +8,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; @@ -23,7 +23,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; @@ -38,7 +38,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; diff --git a/tests/queries/0_stateless/01753_direct_dictionary_simple_key.sql b/tests/queries/0_stateless/01753_direct_dictionary_simple_key.sql index 86af09f391d..93ed3f93c4e 100644 --- a/tests/queries/0_stateless/01753_direct_dictionary_simple_key.sql +++ b/tests/queries/0_stateless/01753_direct_dictionary_simple_key.sql @@ -41,7 +41,7 @@ SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple SELECT 'dictHas'; SELECT dictHas('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4; SELECT 'select all values as input stream'; -SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes; +SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes ORDER BY ALL; DROP DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes; DROP TABLE 01753_dictionary_db.simple_key_simple_attributes_source_table; diff --git a/tests/queries/0_stateless/02372_analyzer_join.reference b/tests/queries/0_stateless/02372_analyzer_join.reference index eefcb1e50dc..9204dded262 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.reference +++ b/tests/queries/0_stateless/02372_analyzer_join.reference @@ -5,63 +5,63 @@ JOIN INNER SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 SELECT '--'; -- SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_1_Value_1 Join_2_Value_1 -SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } -SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 SELECT 'JOIN expression aliases'; JOIN expression aliases -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 SELECT '--'; -- -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 SELECT 'JOIN LEFT'; @@ -69,75 +69,75 @@ JOIN LEFT SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 SELECT '--'; -- SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_1_Value_1 Join_2_Value_1 Join_1_Value_2 -SELECT id FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } -SELECT value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT id FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 SELECT 'JOIN expression aliases'; JOIN expression aliases -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 SELECT '--'; -- -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 @@ -146,182 +146,182 @@ JOIN RIGHT SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 SELECT '--'; -- SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_1_Value_1 Join_2_Value_1 - Join_2_Value_3 -SELECT id FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } -SELECT value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT id FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 1 Join_2_Value_1 0 3 Join_2_Value_3 +0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 1 Join_2_Value_1 0 3 Join_2_Value_3 +0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; 0 1 Join_2_Value_1 0 3 Join_2_Value_3 +0 Join_1_Value_0 0 Join_2_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT 'JOIN expression aliases'; JOIN expression aliases -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT '--'; -- -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT 'JOIN FULL'; JOIN FULL SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 SELECT '--'; -- SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_1_Value_1 Join_2_Value_1 Join_1_Value_2 - Join_2_Value_3 -SELECT id FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } -SELECT value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT id FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL; +0 1 Join_2_Value_1 +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 -0 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; +0 1 Join_2_Value_1 +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 -0 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; +0 1 Join_2_Value_1 +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 2 Join_1_Value_2 0 -0 1 Join_2_Value_1 -0 3 Join_2_Value_3 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 -0 3 Join_2_Value_3 SELECT 'JOIN expression aliases'; JOIN expression aliases -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 -0 3 Join_2_Value_3 SELECT '--'; -- -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL; +0 3 Join_2_Value_3 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 2 Join_1_Value_2 0 -0 3 Join_2_Value_3 SELECT 'First JOIN INNER second JOIN INNER'; First JOIN INNER second JOIN INNER SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; @@ -329,48 +329,48 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id -INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT 'First JOIN INNER second JOIN LEFT'; @@ -379,14 +379,14 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; @@ -394,48 +394,48 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id -LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT 'First JOIN INNER second JOIN RIGHT'; @@ -444,159 +444,159 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT 'First JOIN INNER second JOIN FULL'; First JOIN INNER second JOIN FULL SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id -FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT 'First JOIN LEFT second JOIN INNER'; First JOIN LEFT second JOIN INNER SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -604,7 +604,7 @@ SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -613,7 +613,7 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 @@ -621,7 +621,7 @@ SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -629,20 +629,20 @@ SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -650,7 +650,7 @@ SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -658,7 +658,7 @@ SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -668,7 +668,7 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -676,7 +676,7 @@ SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -685,7 +685,7 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 @@ -693,7 +693,7 @@ SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -701,7 +701,7 @@ SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 @@ -709,7 +709,7 @@ SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 @@ -718,7 +718,7 @@ JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -726,7 +726,7 @@ SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -734,7 +734,7 @@ SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -744,184 +744,184 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT 'First JOIN LEFT second JOIN FULL'; First JOIN LEFT second JOIN FULL SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 0 4 Join_3_Value_4 -0 0 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT 'First JOIN RIGHT second JOIN INNER'; First JOIN RIGHT second JOIN INNER SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; @@ -929,48 +929,48 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT 'First JOIN RIGHT second JOIN LEFT'; @@ -979,246 +979,246 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_2_Value_3 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 1 Join_2_Value_1 0 0 3 Join_2_Value_3 0 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_2_Value_3 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 SELECT 'First JOIN RIGHT second JOIN RIGHT'; First JOIN RIGHT second JOIN RIGHT SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT 'First JOIN RIGHT second JOIN FULL'; First JOIN RIGHT second JOIN FULL SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_2_Value_3 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 -0 1 Join_2_Value_1 0 -0 0 4 Join_3_Value_4 +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_3 0 +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_2_Value_3 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id -FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT 'First JOIN FULL second JOIN INNER'; First JOIN FULL second JOIN INNER SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -1226,7 +1226,7 @@ SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -1235,7 +1235,7 @@ SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 @@ -1243,7 +1243,7 @@ SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -1251,20 +1251,20 @@ SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 @@ -1272,7 +1272,7 @@ SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -1280,7 +1280,7 @@ SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id -INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 @@ -1290,265 +1290,265 @@ SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_2_Value_3 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 3 Join_2_Value_3 0 -0 1 Join_2_Value_1 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_3 0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_2_Value_3 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id -LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 SELECT 'First JOIN FULL second JOIN RIGHT'; First JOIN FULL second JOIN RIGHT SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; 0 0 1 Join_3_Value_1 0 0 4 Join_3_Value_4 +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id -RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 0 4 Join_3_Value_4 SELECT 'First JOIN FULL second JOIN FULL'; First JOIN FULL second JOIN FULL SELECT 'JOIN ON without conditions'; JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 -0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 SELECT '--'; -- SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; + Join_3_Value_4 + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_2_Value_3 - Join_3_Value_4 SELECT 'JOIN ON with conditions'; JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 3 Join_2_Value_3 0 -0 1 Join_2_Value_1 0 -0 0 4 Join_3_Value_4 -0 0 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 2 Join_1_Value_2 0 0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_3 0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_4 SELECT 'JOIN multiple clauses'; JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; + Join_3_Value_4 + Join_2_Value_3 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 Join_1_Value_2 Join_3_Value_0 - Join_2_Value_3 - Join_3_Value_4 SELECT 'JOIN expression aliases'; JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id -FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; +0 0 4 Join_3_Value_4 +0 3 Join_2_Value_3 0 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 2 Join_1_Value_2 0 0 Join_3_Value_0 -0 3 Join_2_Value_3 0 -0 0 4 Join_3_Value_4 diff --git a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 index facf4dc018b..45ae63b9a49 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 +++ b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 @@ -45,59 +45,59 @@ SELECT 'JOIN {{ join_type }}'; SELECT 'JOIN ON without conditions'; SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; -SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } -SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER } +SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER } SELECT 'JOIN ON with conditions'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL; SELECT 'JOIN multiple clauses'; SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL; SELECT 'JOIN expression aliases'; -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL; SELECT '--'; -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL; {% endfor %} @@ -110,56 +110,56 @@ SELECT 'JOIN ON without conditions'; SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value FROM test_table_join_1 {{ first_join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id -{{ second_join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +{{ second_join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; SELECT '--'; SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; SELECT '--'; SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL; SELECT 'JOIN ON with conditions'; SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL; SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL; SELECT 'JOIN multiple clauses'; SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL; SELECT 'JOIN expression aliases'; SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL; SELECT '--'; SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id -{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL; {% endfor %} {% endfor %} diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference index 3722c23e4a0..2b3671e1ea6 100644 --- a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference @@ -1,27 +1,27 @@ -- { echoOn } SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String SELECT '--'; -- SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) 2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String) SELECT '--'; -- SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String SELECT '--'; -- SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) 2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String) @@ -30,14 +30,14 @@ SELECT '--'; -- SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; 0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String SELECT '--'; -- SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; 0 UInt64 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 UInt64 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) 2 UInt64 2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String) @@ -45,7 +45,7 @@ SELECT '--'; -- SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; 0 UInt64 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String 1 UInt64 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String 3 UInt64 \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String @@ -53,7 +53,7 @@ SELECT '--'; -- SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; 0 Nullable(UInt64) 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 Nullable(UInt64) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) 2 Nullable(UInt64) 2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String) diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql index db7895084e8..bcec6d178a8 100644 --- a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql @@ -26,46 +26,46 @@ INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); -- { echoOn } SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL; SELECT '--'; SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; SELECT '--'; SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; SELECT '--'; SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; SELECT '--'; SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL; -- { echoOff } diff --git a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql index 127c6fee07d..dcb7c09a973 100644 --- a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql +++ b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql @@ -1,4 +1,8 @@ SYSTEM FLUSH LOGS; SELECT 'Column ' || name || ' from table ' || concat(database, '.', table) || ' should have a comment' FROM system.columns -WHERE (database = 'system') AND (comment = '') AND (table NOT ILIKE '%_log_%') AND (table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log')) AND (default_kind != 'ALIAS'); +WHERE (database = 'system') AND + (comment = '') AND + (table NOT ILIKE '%_log_%') AND + (table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log', 'filesystem_read_prefetches_log')) AND + (default_kind != 'ALIAS'); From a493e5c8e7e885cc8b66626ebf2911a6e3387b78 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 22 Jul 2024 11:05:37 +0000 Subject: [PATCH 072/321] Followup #66725 --- .../Passes/LogicalExpressionOptimizerPass.cpp | 11 +++++++++-- .../02911_join_on_nullsafe_optimization.reference | 2 ++ .../02911_join_on_nullsafe_optimization.sql | 2 ++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 5c68bca3a6e..e136440556f 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -68,10 +68,13 @@ QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes) return nullptr; } -/// Checks if the node is combination of isNull and notEquals functions of two the same arguments +/// Checks if the node is combination of isNull and notEquals functions of two the same arguments: +/// [ (a <> b AND) ] (a IS NULL) AND (b IS NULL) bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, QueryTreeNodePtr & rhs) { QueryTreeNodePtrWithHashSet all_arguments; + QueryTreeNodePtrWithHashSet is_null_arguments; + for (const auto & node : nodes) { const auto * func_node = node->as(); @@ -80,7 +83,11 @@ bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, const auto & arguments = func_node->getArguments().getNodes(); if (func_node->getFunctionName() == "isNull" && arguments.size() == 1) + { all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0])); + is_null_arguments.insert(QueryTreeNodePtrWithHash(arguments[0])); + } + else if (func_node->getFunctionName() == "notEquals" && arguments.size() == 2) { if (arguments[0]->isEqual(*arguments[1])) @@ -95,7 +102,7 @@ bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, return false; } - if (all_arguments.size() != 2) + if (all_arguments.size() != 2 || is_null_arguments.size() != 2) return false; lhs = all_arguments.begin()->node; diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 31a1cda18e7..8f194b4ffde 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -39,6 +39,8 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N 2 2 2 2 3 3 3 33 \N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND t2.x <> t1.x ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t2.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION } -- aliases defined in the join condition are valid -- FIXME(@vdimir) broken query formatting for the following queries: -- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index f739259caf9..18cb303a54a 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -35,6 +35,8 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND t2.x <> t1.x ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t2.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION } -- aliases defined in the join condition are valid -- FIXME(@vdimir) broken query formatting for the following queries: 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 073/321] 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 a3dbf87df6a7b3105fdbc79cafe0b2ec2fa547d5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:19:16 +0200 Subject: [PATCH 074/321] Update convertFieldToType.cpp --- src/Interpreters/convertFieldToType.cpp | 38 ++++++++++++------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9ee214f4415..b92cbae7b09 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -384,25 +384,25 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { const auto & element_type = *(type_tuple->getElements()[i]); res[i] = convertFieldToType(src_tuple[i], element_type); - if (!res[i].isNull() || canContainNull(element_type)) - continue; - - /* - * Either the source element was Null, or the conversion did not - * succeed, because the source and the requested types of the - * element are compatible, but the value is not convertible - * (e.g. trying to convert -1 from Int8 to UInt8). In these - * cases, consider the whole tuple also compatible but not - * convertible. According to the specification of this function, - * we must return Null in this case. - * - * The following elements might be not even compatible, so it - * makes sense to check them to detect user errors. Remember - * that there is an unconvertible element, and try to process - * the remaining ones. The convertFieldToType for each element - * will throw if it detects incompatibility. - */ - have_unconvertible_element = true; + if (res[i].isNull() && !canContainNull(element_type)) + { + /* + * Either the source element was Null, or the conversion did not + * succeed, because the source and the requested types of the + * element are compatible, but the value is not convertible + * (e.g. trying to convert -1 from Int8 to UInt8). In these + * cases, consider the whole tuple also compatible but not + * convertible. According to the specification of this function, + * we must return Null in this case. + * + * The following elements might be not even compatible, so it + * makes sense to check them to detect user errors. Remember + * that there is an unconvertible element, and try to process + * the remaining ones. The convertFieldToType for each element + * will throw if it detects incompatibility. + */ + have_unconvertible_element = true; + } } return have_unconvertible_element ? Field(Null()) : Field(res); From 660530c611000f5eb8875c640d5aed196315a187 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 17:10:39 +0200 Subject: [PATCH 075/321] Fix tidy --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index a36a8b031b4..377f6b36888 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -258,10 +258,9 @@ std::optional ReadBufferFromAzureBlobStorage::tryGetFileSize() if (!blob_client) blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); - if (file_size.has_value()) - return *file_size; + if (!file_size) + file_size = blob_client->GetProperties().Value.BlobSize; - file_size = blob_client->GetProperties().Value.BlobSize; return *file_size; } From eb519c501622af6a6df6f3370b6209c92c2d4c20 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 20 Jul 2024 15:01:47 +0200 Subject: [PATCH 076/321] 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 4b0b5b7f2dd66ecfd9cb0b533166a229e501cd52 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jul 2024 18:53:26 +0200 Subject: [PATCH 077/321] groupArrayIntersect: Fix internal name --- .../AggregateFunctionGroupArrayIntersect.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 20acda213da..591422adc57 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -69,7 +69,7 @@ public: : IAggregateFunctionDataHelper, AggregateFunctionGroupArrayIntersect>({argument_type}, parameters_, result_type_) {} - String getName() const override { return "GroupArrayIntersect"; } + String getName() const override { return "groupArrayIntersect"; } bool allocatesMemoryInArena() const override { return false; } @@ -213,7 +213,7 @@ public: : IAggregateFunctionDataHelper>({input_data_type_}, parameters_, result_type_) , input_data_type(result_type_) {} - String getName() const override { return "GroupArrayIntersect"; } + String getName() const override { return "groupArrayIntersect"; } bool allocatesMemoryInArena() const override { return true; } From 468bd551c64057fb056ed5434e68c26ded062a84 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jul 2024 19:02:20 +0200 Subject: [PATCH 078/321] Fix includes --- .../AggregateFunctionGroupArrayIntersect.cpp | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 591422adc57..1529cd5899a 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -1,12 +1,12 @@ -#include -#include - #include #include #include #include -#include +#include +#include +#include +#include #include #include @@ -15,18 +15,14 @@ #include #include -#include -#include - #include #include -#include #include -#include -#include -#include -#include +#include +#include + +#include namespace DB @@ -240,7 +236,7 @@ public: { const char * begin = nullptr; StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin); - assert(serialized.data != nullptr); + chassert(serialized.data != nullptr); set.emplace(SerializedKeyHolder{serialized, *arena}, it, inserted); } } @@ -260,7 +256,7 @@ public: { const char * begin = nullptr; StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin); - assert(serialized.data != nullptr); + chassert(serialized.data != nullptr); it = set.find(serialized); if (it != nullptr) From 078bddd65e0978c3feb2f8d77008f3c23bbc136f Mon Sep 17 00:00:00 2001 From: joelynch Date: Mon, 22 Jul 2024 18:55:23 +0200 Subject: [PATCH 079/321] Let window view take comments --- src/Parsers/ParserCreateQuery.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 014dc7bd3bf..3e908ea3602 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -1176,6 +1176,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (!select_p.parse(pos, select, expected)) return false; + auto comment = parseComment(pos, expected); auto query = std::make_shared(); node = query; @@ -1194,6 +1195,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->children.push_back(query->database); if (query->table) query->children.push_back(query->table); + if (comment) + query->set(query->comment, comment); if (to_table) query->to_table_id = to_table->as()->getTableId(); From 240f04561e1ebca2fb1823a6ee34d4f2c15d1e66 Mon Sep 17 00:00:00 2001 From: joelynch Date: Mon, 22 Jul 2024 19:07:46 +0200 Subject: [PATCH 080/321] Fix docs for COMMENT clause --- docs/en/sql-reference/statements/create/table.md | 7 +------ docs/en/sql-reference/statements/create/view.md | 8 +++++++- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index b866d0b9f5f..9c8984d698f 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -21,7 +21,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2], ... ) ENGINE = engine - COMMENT 'comment for table' + [COMMENT 'comment for table'] ``` Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine. @@ -626,11 +626,6 @@ SELECT * FROM base.t1; You can add a comment to the table when you creating it. -:::note -The comment clause is supported by all table engines except [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) and [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md). -::: - - **Syntax** ``` sql diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 1fabb6d8cc7..2931f7020fb 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -16,6 +16,7 @@ Syntax: CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... +[COMMENT 'comment'] ``` Normal views do not store any data. They just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause. @@ -57,6 +58,7 @@ SELECT * FROM view(column1=value1, column2=value2 ...) CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... +[COMMENT 'comment'] ``` :::tip @@ -161,6 +163,7 @@ RANDOMIZE FOR interval DEPENDS ON [db.]name [, [db.]name [, ...]] [TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY] AS SELECT ... +[COMMENT 'comment'] ``` where `interval` is a sequence of simple intervals: ```sql @@ -267,7 +270,10 @@ This is an experimental feature that may change in backwards-incompatible ways i ::: ``` sql -CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] AS SELECT ... GROUP BY time_window_function +CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] +AS SELECT ... +GROUP BY time_window_function +[COMMENT 'comment'] ``` Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query. From 4a2708658d6d9b3ab0de9eab41d4e97bf04c3523 Mon Sep 17 00:00:00 2001 From: joelynch Date: Mon, 22 Jul 2024 19:08:04 +0200 Subject: [PATCH 081/321] Add more tables to table comment test --- .../0_stateless/01821_table_comment.reference | 8 ++- .../0_stateless/01821_table_comment.sql | 53 +++++++++++++++++-- 2 files changed, 54 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01821_table_comment.reference b/tests/queries/0_stateless/01821_table_comment.reference index 05acabae3d4..cdd87df43d2 100644 --- a/tests/queries/0_stateless/01821_table_comment.reference +++ b/tests/queries/0_stateless/01821_table_comment.reference @@ -1,4 +1,8 @@ -t1 this is a temtorary table +t1 this is a temporary table t2 this is a MergeTree table t3 this is a Log table -CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temtorary table\' +t4 this is a Kafka table +t5 this is a EmbeddedRocksDB table +t6 this is a Executable table +t7 this is a WindowView table +CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temporary table\' diff --git a/tests/queries/0_stateless/01821_table_comment.sql b/tests/queries/0_stateless/01821_table_comment.sql index 4bd71d3e278..32b89af0750 100644 --- a/tests/queries/0_stateless/01821_table_comment.sql +++ b/tests/queries/0_stateless/01821_table_comment.sql @@ -9,7 +9,7 @@ CREATE TABLE t1 `n` Int8 ) ENGINE = Memory -COMMENT 'this is a temtorary table'; +COMMENT 'this is a temporary table'; CREATE TABLE t2 ( @@ -26,14 +26,57 @@ CREATE TABLE t3 ENGINE = Log COMMENT 'this is a Log table'; +CREATE TABLE t4 +( + `n` Int8 +) +ENGINE = Kafka +SETTINGS + kafka_broker_list = 'localhost:10000', + kafka_topic_list = 'test', + kafka_group_name = 'test', + kafka_format = 'JSONEachRow' +COMMENT 'this is a Kafka table'; + +CREATE TABLE t5 +( + `n` Int8 +) +ENGINE = EmbeddedRocksDB +PRIMARY KEY n +COMMENT 'this is a EmbeddedRocksDB table'; + +CREATE TABLE t6 +( + `n` Int8 +) +ENGINE = Executable('script.py', TabSeparated) +COMMENT 'this is a Executable table'; + +SET allow_experimental_window_view = 1; +-- New analyzer doesn't support WindowView tables +SET allow_experimental_analyzer = 0; + +CREATE WINDOW VIEW t7 +( + `n` Int8 +) +ENGINE MergeTree +ORDER BY n +AS SELECT 1 +GROUP BY tumble(now(), toIntervalDay('1')) +COMMENT 'this is a WindowView table'; + +SET allow_experimental_analyzer = 1; + SELECT name, comment FROM system.tables -WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name; +WHERE name IN ('t1', 't2', 't3', 't4', 't5', 't6', 't7') + AND database = currentDatabase() order by name; SHOW CREATE TABLE t1; -DROP TABLE t1; -DROP TABLE t2; -DROP TABLE t3; +DROP TABLE t1, t2, t3, t4, t5, t6; +DROP VIEW t7; From b1029fbd671310a4c8d48070f87d84f33f8842fd Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Mon, 22 Jul 2024 22:02:54 +0330 Subject: [PATCH 082/321] Fixed the style issue --- src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index e217d93975d..f51a7a913b8 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -37,6 +37,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int BAD_QUERY_PARAMETER; + extern const int QUERY_NOT_ALLOWED; } namespace From 155b28227972fe2f33dac98d3c471c555637d246 Mon Sep 17 00:00:00 2001 From: joelynch Date: Mon, 22 Jul 2024 21:30:40 +0200 Subject: [PATCH 083/321] This cannot be fasttest because it uses Kafka engine --- tests/queries/0_stateless/01821_table_comment.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01821_table_comment.sql b/tests/queries/0_stateless/01821_table_comment.sql index 32b89af0750..4946e46d37a 100644 --- a/tests/queries/0_stateless/01821_table_comment.sql +++ b/tests/queries/0_stateless/01821_table_comment.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-fasttest DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; 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 084/321] =?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 8fb560d2575c121b252ab1e6d8e13f9486dc2b38 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Tue, 23 Jul 2024 07:08:58 +0000 Subject: [PATCH 085/321] add replica sync --- tests/integration/test_broken_projections/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index e198f98e4c5..a565f47449f 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -433,6 +433,7 @@ def test_broken_ignored_replicated(cluster): check(node, table_name, 1) create_table(node, table_name2, 2, table_name) + node.query(f"system sync replica {table_name}") check(node, table_name2, 1) break_projection(node, table_name, "proj1", "all_0_0_0", "data") From d7ffbab7c4eab820b303bd80f6b52e856f4e1d47 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Jul 2024 09:23:18 +0200 Subject: [PATCH 086/321] Add test from #66378 Co-authored-by: Azat Khuzhin --- src/Parsers/ASTTablesInSelectQuery.cpp | 1 + .../0_stateless/03204_format_join_on.reference | 4 ++++ tests/queries/0_stateless/03204_format_join_on.sh | 15 +++++++++++++++ 3 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03204_format_join_on.reference create mode 100644 tests/queries/0_stateless/03204_format_join_on.sh diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index dbb2a008bae..b6d42513aa7 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -235,6 +235,7 @@ void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatS else if (on_expression) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); + /// If there is an alias for the whole expression parens should be added, otherwise it will be invalid syntax bool on_has_alias = !on_expression->tryGetAlias().empty(); if (on_has_alias) settings.ostr << "("; diff --git a/tests/queries/0_stateless/03204_format_join_on.reference b/tests/queries/0_stateless/03204_format_join_on.reference new file mode 100644 index 00000000000..846f36fcca4 --- /dev/null +++ b/tests/queries/0_stateless/03204_format_join_on.reference @@ -0,0 +1,4 @@ +SELECT * FROM t1 INNER JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2) +SELECT * FROM t1 INNER JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2) +SELECT * FROM t1 INNER JOIN t2 ON (t1.x = t2.x) AND ((t1.x IS NULL) AS e2) +SELECT * FROM t1 INNER JOIN t2 ON t1.x = t2.x diff --git a/tests/queries/0_stateless/03204_format_join_on.sh b/tests/queries/0_stateless/03204_format_join_on.sh new file mode 100644 index 00000000000..87b0afac042 --- /dev/null +++ b/tests/queries/0_stateless/03204_format_join_on.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# regression for the JOIN ON alias for the whole expression +phase1="$($CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2)")" +echo "$phase1" +# phase 2 +$CLICKHOUSE_FORMAT --oneline --query "$phase1" + +# other test cases +$CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x) AND (t1.x IS NULL AS e2)" +$CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON t1.x = t2.x" From b6ad57aa37f01ed4d101bd059b04222f361245ff Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 22 Jul 2024 19:25:34 +0200 Subject: [PATCH 087/321] Stateless tests: change status for failed tests in case of server crash --- .../util/process_functional_tests_result.py | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 4442c9d7d9e..dbe50eeade0 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -12,6 +12,7 @@ UNKNOWN_SIGN = "[ UNKNOWN " SKIPPED_SIGN = "[ SKIPPED " HUNG_SIGN = "Found hung queries in processlist" SERVER_DIED_SIGN = "Server died, terminating all processes" +SERVER_DIED_SIGN2 = "Server does not respond to health check" DATABASE_SIGN = "Database: " SUCCESS_FINISH_SIGNS = ["All tests have finished", "No tests were run"] @@ -43,7 +44,7 @@ def process_test_log(log_path, broken_tests): if HUNG_SIGN in line: hung = True break - if SERVER_DIED_SIGN in line: + if SERVER_DIED_SIGN in line or SERVER_DIED_SIGN2 in line: server_died = True if RETRIES_SIGN in line: retries = True @@ -111,12 +112,12 @@ def process_test_log(log_path, broken_tests): # Python does not support TSV, so we have to escape '\t' and '\n' manually # and hope that complex escape sequences will not break anything test_results = [ - ( + [ test[0], test[1], test[2], "".join(test[3])[:4096].replace("\t", "\\t").replace("\n", "\\n"), - ) + ] for test in test_results ] @@ -170,18 +171,24 @@ def process_result(result_path, broken_tests): if hung: description = "Some queries hung, " state = "failure" - test_results.append(("Some queries hung", "FAIL", "0", "")) + test_results.append(["Some queries hung", "FAIL", "0", ""]) elif server_died: description = "Server died, " state = "failure" - test_results.append(("Server died", "FAIL", "0", "")) + # When ClickHouse server crashes, some tests are still running + # and fail because they cannot connect to server + for result in test_results: + if result[1] == "FAIL": + result[1] = "SERVER_DIED" + + test_results.append(["Server died", "FAIL", "0", ""]) elif not success_finish: description = "Tests are not finished, " state = "failure" - test_results.append(("Tests are not finished", "FAIL", "0", "")) + test_results.append(["Tests are not finished", "FAIL", "0", ""]) elif retries: description = "Some tests restarted, " - test_results.append(("Some tests restarted", "SKIPPED", "0", "")) + test_results.append(["Some tests restarted", "SKIPPED", "0", ""]) else: description = "" @@ -233,11 +240,12 @@ if __name__ == "__main__": # sort by status then by check name order = { "FAIL": 0, - "Timeout": 1, - "NOT_FAILED": 2, - "BROKEN": 3, - "OK": 4, - "SKIPPED": 5, + "SERVER_DIED": 1, + "Timeout": 2, + "NOT_FAILED": 3, + "BROKEN": 4, + "OK": 5, + "SKIPPED": 6, } return order.get(item[1], 10), str(item[0]), item[1] From 492dab5e5d4fb775d8f3551e990fc078929c2bd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Jul 2024 01:52:01 +0200 Subject: [PATCH 088/321] Update tests/queries/0_stateless/02992_all_columns_should_have_comment.sql --- .../0_stateless/02992_all_columns_should_have_comment.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql index dcb7c09a973..0d34b033354 100644 --- a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql +++ b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql @@ -3,6 +3,6 @@ SELECT 'Column ' || name || ' from table ' || concat(database, '.', table) || ' FROM system.columns WHERE (database = 'system') AND (comment = '') AND - (table NOT ILIKE '%_log_%') AND + (table NOT ILIKE '%\_log\_%') AND (table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log', 'filesystem_read_prefetches_log')) AND (default_kind != 'ALIAS'); From a000f8f8a13598d8c4dd24043d5029d7a0158ace Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 23 Jul 2024 11:27:16 +0200 Subject: [PATCH 089/321] 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 090/321] 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 86ff4e8b73d99daa5239104a0223271411949b4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Jul 2024 12:39:24 +0200 Subject: [PATCH 091/321] groupArrayIntersect: Fix serialization bug --- .../AggregateFunctionGroupArrayIntersect.cpp | 4 +- ...roupArrayIntersect_serialization.reference | 12 ++++++ ...3208_groupArrayIntersect_serialization.sql | 41 +++++++++++++++++++ 3 files changed, 55 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference create mode 100644 tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 1529cd5899a..38f2fcb9fb9 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -47,7 +47,7 @@ struct AggregateFunctionGroupArrayIntersectData }; -/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types. +/// Puts all values to the hash set. Returns an array of unique values present in all inputs. Implemented for numeric types. template class AggregateFunctionGroupArrayIntersect : public IAggregateFunctionDataHelper, AggregateFunctionGroupArrayIntersect> @@ -154,7 +154,7 @@ public: set.reserve(size); for (size_t i = 0; i < size; ++i) { - int key; + T key; readIntBinary(key, buf); set.insert(key); } diff --git a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference new file mode 100644 index 00000000000..c3b6e0cd5b7 --- /dev/null +++ b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference @@ -0,0 +1,12 @@ +010101 AggregateFunction(groupArrayIntersect, Array(UInt8)) +[1] +1 [2,4,6,8,10] +2 [2,4,6,8,10] +3 [2,4,6,8,10] +5 [2,6,10] +6 [10] +7 [] +a [(['2','4','6','8','10'])] +b [(['2','4','6','8','10'])] +c [(['2','4','6','8','10'])] +d [] diff --git a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql new file mode 100644 index 00000000000..e05f78a4051 --- /dev/null +++ b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql @@ -0,0 +1,41 @@ +SELECT hex(groupArrayIntersectState([1]) AS a), toTypeName(a); +SELECT finalizeAggregation(CAST(unhex('010101'), 'AggregateFunction(groupArrayIntersect, Array(UInt8))')); + +DROP TABLE IF EXISTS grouparray; +CREATE TABLE grouparray +( + `v` AggregateFunction(groupArrayIntersect, Array(UInt8)) +) +ENGINE = Log; + +INSERT INTO grouparray Select groupArrayIntersectState([2, 4, 6, 8, 10]::Array(UInt8)); +SELECT '1', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; +INSERT INTO grouparray Select groupArrayIntersectState([2, 4, 6, 8, 10]::Array(UInt8)); +SELECT '2', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; +INSERT INTO grouparray Select groupArrayIntersectState([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]::Array(UInt8)); +SELECT '3', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; +INSERT INTO grouparray Select groupArrayIntersectState([2, 6, 10]::Array(UInt8)); +SELECT '5', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; +INSERT INTO grouparray Select groupArrayIntersectState([10]::Array(UInt8)); +SELECT '6', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; +INSERT INTO grouparray Select groupArrayIntersectState([]::Array(UInt8)); +SELECT '7', arraySort(groupArrayIntersectMerge(v)) FROM grouparray; + +DROP TABLE IF EXISTS grouparray; + + +DROP TABLE IF EXISTS grouparray_string; +CREATE TABLE grouparray_string +( + `v` AggregateFunction(groupArrayIntersect, Array(Tuple(Array(String)))) +) +ENGINE = Log; + +INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10'])]); +SELECT 'a', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; +INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10']), tuple(['2', '4', '6', '8', '10'])]); +SELECT 'b', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; +INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10']), tuple(['2', '4', '6', '8', '10', '14'])]); +SELECT 'c', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; +INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10', '20']), tuple(['2', '4', '6', '8', '10', '14'])]); +SELECT 'd', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; From 0256dba672bd23302b845f7d78f4663e3c633140 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Jul 2024 13:20:41 +0200 Subject: [PATCH 092/321] Make 02987_group_array_intersect smaller --- .../0_stateless/02987_group_array_intersect.reference | 8 ++++---- .../0_stateless/02987_group_array_intersect.sql | 10 +++++----- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02987_group_array_intersect.reference b/tests/queries/0_stateless/02987_group_array_intersect.reference index 7ec64a889f5..ec4d07742af 100644 --- a/tests/queries/0_stateless/02987_group_array_intersect.reference +++ b/tests/queries/0_stateless/02987_group_array_intersect.reference @@ -8,12 +8,12 @@ [1,4,5] [] [] -1000000 -999999 +100000 +99999 [9] ['a','c'] -1000000 -999999 +50000 +49999 ['1'] [] ['2023-01-01 00:00:00'] diff --git a/tests/queries/0_stateless/02987_group_array_intersect.sql b/tests/queries/0_stateless/02987_group_array_intersect.sql index 321e860b0a8..15acd0ca900 100644 --- a/tests/queries/0_stateless/02987_group_array_intersect.sql +++ b/tests/queries/0_stateless/02987_group_array_intersect.sql @@ -39,15 +39,15 @@ DROP TABLE test_numbers; DROP TABLE IF EXISTS test_big_numbers_sep; CREATE TABLE test_big_numbers_sep (a Array(Int64)) engine=MergeTree ORDER BY a; -INSERT INTO test_big_numbers_sep SELECT array(number) FROM numbers_mt(1000000); +INSERT INTO test_big_numbers_sep SELECT array(number) FROM numbers_mt(100000); SELECT groupArrayIntersect(*) FROM test_big_numbers_sep; DROP TABLE test_big_numbers_sep; DROP TABLE IF EXISTS test_big_numbers; CREATE TABLE test_big_numbers (a Array(Int64)) engine=MergeTree ORDER BY a; -INSERT INTO test_big_numbers SELECT range(1000000); +INSERT INTO test_big_numbers SELECT range(100000); SELECT length(groupArrayIntersect(*)) FROM test_big_numbers; -INSERT INTO test_big_numbers SELECT range(999999); +INSERT INTO test_big_numbers SELECT range(99999); SELECT length(groupArrayIntersect(*)) FROM test_big_numbers; INSERT INTO test_big_numbers VALUES ([9]); SELECT groupArrayIntersect(*) FROM test_big_numbers; @@ -63,9 +63,9 @@ DROP TABLE test_string; DROP TABLE IF EXISTS test_big_string; CREATE TABLE test_big_string (a Array(String)) engine=MergeTree ORDER BY a; -INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(1000000); +INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(50000); SELECT length(groupArrayIntersect(*)) FROM test_big_string; -INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(999999); +INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(49999); SELECT length(groupArrayIntersect(*)) FROM test_big_string; INSERT INTO test_big_string VALUES (['1']); SELECT groupArrayIntersect(*) FROM test_big_string; From bc9c462155bed43eeb72415660917900ee4e7b58 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jul 2024 15:47:55 +0100 Subject: [PATCH 093/321] 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 094/321] 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 095/321] 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 096/321] 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 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 097/321] 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 098/321] 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 099/321] 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 1cbbbd107712c066bef673e5d4692a6950d9e85e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 23 Jul 2024 17:35:10 +0000 Subject: [PATCH 100/321] 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 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 101/321] 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 102/321] 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 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 103/321] 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 104/321] 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 105/321] 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 106/321] 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 107/321] 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 108/321] 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 109/321] 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 c850fac65276342e0b8694fa00d44dd3269d1abc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 09:05:06 +0200 Subject: [PATCH 110/321] Fix error --- src/Parsers/MySQL/tests/gtest_column_parser.cpp | 11 ++++++----- src/Parsers/ParserCreateQuery.h | 7 ++++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Parsers/MySQL/tests/gtest_column_parser.cpp b/src/Parsers/MySQL/tests/gtest_column_parser.cpp index 21c37e4ee2e..3a9a0690f06 100644 --- a/src/Parsers/MySQL/tests/gtest_column_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_column_parser.cpp @@ -1,13 +1,14 @@ #include #include #include -#include +#include #include #include #include #include #include + using namespace DB; using namespace DB::MySQLParser; @@ -19,8 +20,8 @@ TEST(ParserColumn, AllNonGeneratedColumnOption) "COLUMN_FORMAT FIXED STORAGE MEMORY REFERENCES tbl_name (col_01) CHECK 1"; ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0); EXPECT_EQ(ast->as()->name, "col_01"); - EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); - EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); ASTDeclareOptions * declare_options = ast->as()->column_options->as(); EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 0); @@ -44,8 +45,8 @@ TEST(ParserColumn, AllGeneratedColumnOption) "REFERENCES tbl_name (col_01) CHECK 1 GENERATED ALWAYS AS (1) STORED"; ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0); EXPECT_EQ(ast->as()->name, "col_01"); - EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); - EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); ASTDeclareOptions * declare_options = ast->as()->column_options->as(); EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 1); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 7bd1d1bf588..53a62deb22b 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include + namespace DB { @@ -268,9 +270,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E auto default_function = std::make_shared(); default_function->name = "defaultValueOfTypeName"; default_function->arguments = std::make_shared(); - // Ephemeral columns don't really have secrets but we need to format - // into a String, hence the strange call - default_function->arguments->children.emplace_back(std::make_shared(type->as()->formatForLogging())); + /// Ephemeral columns don't really have secrets but we need to format into a String, hence the strange call + default_function->arguments->children.emplace_back(std::make_shared(type->as()->formatForLogging())); default_expression = default_function; } From c3204fb89577e50ec7ef2c7ddd3c62f913e084f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 09:25:22 +0200 Subject: [PATCH 111/321] Fix error --- src/Parsers/ExpressionElementParsers.cpp | 3 +-- src/Parsers/ExpressionListParsers.cpp | 17 +++++++++++++++++ src/Parsers/ExpressionListParsers.h | 10 ++++++++++ src/Parsers/ParserCreateIndexQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 5 files changed, 31 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index d4fc9a4bc4d..865d07faaa7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -9,8 +9,8 @@ #include #include #include -#include "Parsers/CommonParsers.h" +#include #include #include #include @@ -725,7 +725,6 @@ bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expect function_node->name = "STATISTICS"; function_node->arguments = stat_type; function_node->children.push_back(function_node->arguments); - node = function_node; return true; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f97c042e91e..66817fafa5e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2388,6 +2388,23 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } +bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserIdentifier id_p; + ParserFunction func_p; + + if (ParserFunction(false, false).parse(pos, node, expected)) + return true; + + if (ParserIdentifier().parse(pos, node, expected)) + { + node = makeASTFunction(node->as()->name()); + return true; + } + + return false; +} + const std::vector> ParserExpressionImpl::operators_table { {"->", Operator("lambda", 1, 2, OperatorType::Lambda)}, diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 235d5782630..6ab38416f32 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -144,6 +144,16 @@ protected: }; +/** Similar to ParserFunction (and yields ASTFunction), but can also parse identifiers without braces. + */ +class ParserExpressionWithOptionalArguments : public IParserBase +{ +protected: + const char * getName() const override { return "expression with optional parameters"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** An expression with an infix binary left-associative operator. * For example, a + b - c + d. */ diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 2761c99738b..b815ba60bab 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -21,7 +21,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ParserToken close_p(TokenType::ClosingRoundBracket); ParserOrderByExpressionList order_list_p; - ParserFunction type_p; + ParserExpressionWithOptionalArguments type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 92c0e7b2558..5da6c3a2510 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -180,7 +180,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_granularity(Keyword::GRANULARITY); ParserIdentifier name_p; - ParserFunction type_p; + ParserExpressionWithOptionalArguments type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -240,7 +240,7 @@ bool ParserStatisticsDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_type(Keyword::TYPE); ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); - ParserList types_p(std::make_unique(), std::make_unique(TokenType::Comma), false); + ParserList types_p(std::make_unique(), std::make_unique(TokenType::Comma), false); ASTPtr columns; ASTPtr types; From 5fe78d47bc855867f6431ad06e019b3e0278d0ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:24:13 +0200 Subject: [PATCH 112/321] Compatibility --- src/Parsers/ExpressionListParsers.cpp | 1 + src/Parsers/ParserCreateIndexQuery.cpp | 10 +--------- src/Parsers/ParserCreateQuery.cpp | 10 +--------- 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 66817fafa5e..a9715cec81e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2399,6 +2399,7 @@ bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, if (ParserIdentifier().parse(pos, node, expected)) { node = makeASTFunction(node->as()->name()); + node->as().no_empty_args = true; return true; } diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index b815ba60bab..9ebee4cc852 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -69,15 +69,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { if (!type_p.parse(pos, type, expected)) - { - if (ParserIdentifier().parse(pos, type, expected)) - { - type = makeASTFunction(type->as().name()); - type->as().no_empty_args = true; - } - else - return false; - } + return false; } if (s_granularity.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 5da6c3a2510..bf5523152ac 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -199,15 +199,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!type_p.parse(pos, type, expected)) - { - if (name_p.parse(pos, type, expected)) - { - type = makeASTFunction(type->as().name()); - type->as().no_empty_args = true; - } - else - return false; - } + return false; if (s_granularity.ignore(pos, expected)) { From e5bb485a006d93a9e00736dc37ad90a0a0a47673 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:25:23 +0200 Subject: [PATCH 113/321] Compatibility --- src/Parsers/ExpressionListParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a9715cec81e..d38dc6d5f37 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2399,7 +2399,7 @@ bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, if (ParserIdentifier().parse(pos, node, expected)) { node = makeASTFunction(node->as()->name()); - node->as().no_empty_args = true; + node->as().no_empty_args = true; return true; } From 73fc5c266f3bc254db3882bfa2f9f42db6b2bc87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:37:19 +0200 Subject: [PATCH 114/321] Fix error --- src/DataTypes/DataTypeObject.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 5636a46373f..91b9bfcb2a5 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -4,9 +4,10 @@ #include #include -#include +#include #include + namespace DB { @@ -53,13 +54,13 @@ static DataTypePtr create(const ASTPtr & arguments) ASTPtr schema_argument = arguments->children[0]; bool is_nullable = false; - if (const auto * func = schema_argument->as()) + if (const auto * type = schema_argument->as()) { - if (func->name != "Nullable" || func->arguments->children.size() != 1) + if (type->name != "Nullable" || type->arguments->children.size() != 1) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, - "Expected 'Nullable()' as parameter for type Object (function: {})", func->name); + "Expected 'Nullable()' as parameter for type Object (function: {})", type->name); - schema_argument = func->arguments->children[0]; + schema_argument = type->arguments->children[0]; is_nullable = true; } From 42fe58f0466cf5a74f88c1a48bee9e96540f01e5 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 24 Jul 2024 08:33:00 +0000 Subject: [PATCH 115/321] 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 e46d400f7beba9479dfcd0d63f025268081f625c Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 24 Jul 2024 09:03:05 +0000 Subject: [PATCH 116/321] 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 57a6d281000f0a49116db82e8b0b364990e61970 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 11:17:43 +0200 Subject: [PATCH 117/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 85230957b3f..17a5ed385d4 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -132,6 +132,14 @@ std::optional ReadWriteBufferFromHTTP::tryGetFileSize() { return std::nullopt; } + catch (const NetException &) + { + return std::nullopt; + } + catch (const Poco::Net::NetException &) + { + return std::nullopt; + } } return file_info->file_size; From 0f9ee5c37d68f0877c0bc982c6bb59bf4803f98e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 11:25:08 +0200 Subject: [PATCH 118/321] 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 119/321] 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 120/321] 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 2ea10d99940132828c61457d62a54b77a7a66af2 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 24 Jul 2024 13:13:58 +0200 Subject: [PATCH 121/321] 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 122/321] 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 123/321] 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 124/321] 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 125/321] 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 126/321] 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 127/321] Bump From 01ce22049a76995dc00974618c94af9ccbcc30db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:29:14 +0200 Subject: [PATCH 128/321] Fix tests --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 11 +++++------ .../MySQL/tests/gtest_create_rewritten.cpp | 6 ++---- .../test_postgresql_replica_database_engine_2/test.py | 2 +- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index f73965cfcc8..3917ffb8823 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -17,7 +17,6 @@ #include #include -#include #include #include #include @@ -158,7 +157,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col /// (see git blame for details). auto column_name_and_type = columns_name_and_type.begin(); const auto * declare_column_ast = columns_definition->children.begin(); - for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++) + for (; column_name_and_type != columns_name_and_type.end(); ++column_name_and_type, ++declare_column_ast) { const auto & declare_column = (*declare_column_ast)->as(); String comment; @@ -177,7 +176,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col return columns_description; } -static NamesAndTypesList getNames(const ASTDataType & expr, ContextPtr context, const NamesAndTypesList & columns) +static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, const NamesAndTypesList & columns) { if (expr.arguments->children.empty()) return NamesAndTypesList{}; @@ -221,9 +220,9 @@ static std::tuplechildren.empty()) { diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 6d6077a0295..81e6e6a8761 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -2,12 +2,10 @@ #include -#include #include #include #include #include -#include #include #include #include @@ -26,8 +24,8 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, ContextPtr co context, "test_database", "test_database")[0]; } -static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8() MATERIALIZED 1" - ", `_version` UInt64() MATERIALIZED 1" +static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8 MATERIALIZED 1" + ", `_version` UInt64 MATERIALIZED 1" ", INDEX _version _version TYPE minmax GRANULARITY 1"; TEST(MySQLCreateRewritten, ColumnsDataType) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 5e04c9e4d12..406b50bc486 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -654,7 +654,7 @@ def test_table_override(started_cluster): instance.query(f"SELECT count() FROM {materialized_database}.{table_name}") ) - expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` String,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nPARTITION BY key\\nORDER BY tuple(key)" + expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` String,\\n `_sign` Int8 MATERIALIZED 1,\\n `_version` UInt64 MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nPARTITION BY key\\nORDER BY tuple(key)" assert ( expected == instance.query( From 6efd29144558ded7fb95b36c6c19ee50aee0071f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:37:05 +0200 Subject: [PATCH 129/321] Add a test --- .../03210_inconsistent_formatting_of_data_types.reference | 1 + .../03210_inconsistent_formatting_of_data_types.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference create mode 100755 tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference new file mode 100644 index 00000000000..ccb445a0573 --- /dev/null +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference @@ -0,0 +1 @@ +ALTER TABLE columns_with_multiple_streams MODIFY COLUMN `field1` Nullable(tupleElement(x, 2), UInt8) diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh new file mode 100755 index 00000000000..6cb2d083d71 --- /dev/null +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field1 Nullable(tupleElement(x, 2), UInt8)" | $CLICKHOUSE_FORMAT --oneline From cb7fafd1e6f04a2f29cd77036bb29042b4cfe3f6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 14:39:26 +0200 Subject: [PATCH 130/321] 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 131/321] 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 132/321] 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 133/321] 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 134/321] 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 135/321] 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 136/321] 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 137/321] 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 138/321] Bump From 535c872b39b7f593e9e63e614a78d39cde5dd4d9 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Wed, 24 Jul 2024 15:46:48 +0200 Subject: [PATCH 139/321] 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 140/321] 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 141/321] 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 37c345bb4925095da3e82e3fc3ed27072786d7e7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 24 Jul 2024 16:01:21 +0200 Subject: [PATCH 142/321] rewrite 01171 test --- ..._mv_select_insert_isolation_long.reference | 4 - .../01171_mv_select_insert_isolation_long.sh | 229 ++++++++++++------ 2 files changed, 152 insertions(+), 81 deletions(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference index d8bb9e310e6..e69de29bb2d 100644 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference @@ -1,4 +0,0 @@ -275 0 138 136 0 -275 0 -275 0 138 136 0 -275 0 diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 2ab7f883367..f6850864be5 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-ordinary-database, no-debug +# Tags: long, no-ordinary-database # Test is too heavy, avoid parallel run in Flaky Check # shellcheck disable=SC2119 @@ -7,82 +7,125 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -set -e +set -ue $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tmp"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm SETTINGS old_parts_lifetime=0"; -$CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int16) AS SELECT n*m AS nm FROM src"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt8, nm Int16) ENGINE=MergeTree ORDER BY (x, nm) SETTINGS old_parts_lifetime=0" +$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int32, m Int32, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int32, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm SETTINGS old_parts_lifetime=0"; +$CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int32) AS SELECT n*m AS nm FROM src"; + +$CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt32, nm Int32) ENGINE=MergeTree ORDER BY (x, nm) SETTINGS old_parts_lifetime=0" $CLICKHOUSE_CLIENT --query "INSERT INTO src VALUES (0, 0)" -# some transactions will fail due to constraint -function thread_insert_commit() +function get_now() { - set -e - for i in {1..100}; do - $CLICKHOUSE_CLIENT --multiquery --query " - BEGIN TRANSACTION; - INSERT INTO src VALUES /* ($i, $1) */ ($i, $1); - SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $1))) FORMAT Null; - INSERT INTO src VALUES /* (-$i, $1) */ (-$i, $1); - COMMIT;" 2>&1| grep -Fv "is violated at row" | grep -Fv "Transaction is not in RUNNING state" | grep -F "Received from " ||: - done + date +%s } -function thread_insert_rollback() +is_pid_exist() +{ + local pid=$1 + ps -p $pid > /dev/null +} + +function run_until_deadline_and_at_least_times() { set -e - for _ in {1..100}; do - $CLICKHOUSE_CLIENT --multiquery --query " - BEGIN TRANSACTION; - INSERT INTO src VALUES /* (42, $1) */ (42, $1); - SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$1) != 1) FORMAT Null; - ROLLBACK;" + + local deadline=$1; shift + local min_iterations=$1; shift + local function_to_run=$1; shift + + local started_time=$(get_now) + local i=0 + + while true + do + $function_to_run $i $@ + + [[ $(get_now) -lt $deadline ]] || break + + i=$(($i + 1)) done + + [[ $i -gt $min_iterations ]] || echo "$i/$min_iterations : not enough iterations of $function_to_run has been made from $started_time until $deadline" >&2 +} + +function insert_commit_action() +{ + set -e + + local i=$1; shift + local tag=$1; shift + + # some transactions will fail due to constraint + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES /* ($i, $tag) */ ($i, $tag); + SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $tag))) /* ($i, $tag) */ FORMAT Null; + INSERT INTO src VALUES /* (-$i, $tag) */ (-$i, $tag); + COMMIT; + " 2>&1 \ + | grep -Fv "is violated at row" | grep -Fv "Transaction is not in RUNNING state" | grep -F "Received from " ||: +} + + +function insert_rollback_action() +{ + set -e + + local i=$1; shift + local tag=$1; shift + + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES /* (42, $tag) */ (42, $tag); + SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$tag) != 1) FORMAT Null; + ROLLBACK;" } # make merges more aggressive -function thread_optimize() +function optimize_action() { set -e - while true; do - optimize_query="OPTIMIZE TABLE src" - partition_id=$(( RANDOM % 2 )) - if (( RANDOM % 2 )); then - optimize_query="OPTIMIZE TABLE dst" - partition_id="all" - fi - if (( RANDOM % 2 )); then - optimize_query="$optimize_query PARTITION ID '$partition_id'" - fi - if (( RANDOM % 2 )); then - optimize_query="$optimize_query FINAL" - fi - action="COMMIT" - if (( RANDOM % 4 )); then - action="ROLLBACK" - fi - $CLICKHOUSE_CLIENT --multiquery --query " + optimize_query="OPTIMIZE TABLE src" + partition_id=$(( RANDOM % 2 )) + if (( RANDOM % 2 )); then + optimize_query="OPTIMIZE TABLE dst" + partition_id="all" + fi + if (( RANDOM % 2 )); then + optimize_query="$optimize_query PARTITION ID '$partition_id'" + fi + if (( RANDOM % 2 )); then + optimize_query="$optimize_query FINAL" + fi + action="COMMIT" + if (( RANDOM % 4 )); then + action="ROLLBACK" + fi + + $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - $optimize_query; + $optimize_query; $action; - " 2>&1| grep -Fv "already exists, but it will be deleted soon" | grep -F "Received from " ||: - sleep 0.$RANDOM; - done + " 2>&1 \ + | grep -Fv "already exists, but it will be deleted soon" | grep -F "Received from " ||: + + sleep 0.$RANDOM; } -function thread_select() +function select_action() { set -e - while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + + $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; @@ -90,14 +133,13 @@ function thread_select() SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; COMMIT;" - done } -function thread_select_insert() +function select_insert_action() { set -e - while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + + $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null; INSERT INTO tmp SELECT 1, n*m FROM src; @@ -110,36 +152,69 @@ function thread_select_insert() SELECT throwIf(1 != (SELECT countDistinct(arr) FROM (SELECT x, arraySort(groupArray(nm)) AS arr FROM tmp WHERE x!=4 GROUP BY x))) FORMAT Null; SELECT throwIf((SELECT count(), sum(nm) FROM tmp WHERE x=4) != (SELECT count(), sum(nm) FROM tmp WHERE x!=4)) FORMAT Null; ROLLBACK;" - done } -thread_insert_commit 1 & PID_1=$! -thread_insert_commit 2 & PID_2=$! -thread_insert_rollback 3 & PID_3=$! +MAIN_TIME_PART=400 +SECOND_TIME_PART=30 +WAIT_FINISH=60 +LAST_TIME_GAP=10 -thread_optimize & PID_4=$! -thread_select & PID_5=$! -thread_select_insert & PID_6=$! -sleep 0.$RANDOM; -thread_select & PID_7=$! -thread_select_insert & PID_8=$! +if [[ $((MAIN_TIME_PART + SECOND_TIME_PART + WAIT_FINISH + LAST_TIME_GAP)) -ge 600 ]]; then + echo "time sttings are wrong" 2>&1 + exit 1 +fi -wait $PID_1 && wait $PID_2 && wait $PID_3 -kill -TERM $PID_4 -kill -TERM $PID_5 -kill -TERM $PID_6 -kill -TERM $PID_7 -kill -TERM $PID_8 -wait -wait_for_queries_to_finish 40 +START_TIME=$(get_now) +STOP_TIME=$((START_TIME + MAIN_TIME_PART)) +SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) +MIN_ITERATIONS=50 + +run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 1 & PID_1=$! +run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 2 & PID_2=$! +run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_rollback_action 3 & PID_3=$! + +run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS optimize_action & PID_4=$! +run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_action & PID_5=$! +run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_insert_action & PID_6=$! +sleep 0.$RANDOM +run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_action & PID_7=$! +run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_insert_action & PID_8=$! + +wait $PID_1 || echo "insert_commit_action has failed with status $?" 2>&1 +wait $PID_2 || echo "second insert_commit_action has failed with status $?" 2>&1 +wait $PID_3 || echo "insert_rollback_action has failed with status $?" 2>&1 + +is_pid_exist $PID_4 || echo "optimize_action is not running" 2>&1 +is_pid_exist $PID_5 || echo "select_action is not running" 2>&1 +is_pid_exist $PID_6 || echo "select_insert_action is not running" 2>&1 +is_pid_exist $PID_7 || echo "second select_action is not running" 2>&1 +is_pid_exist $PID_8 || echo "second select_insert_action is not running" 2>&1 + +wait $PID_4 || echo "optimize_action has failed with status $?" 2>&1 +wait $PID_5 || echo "select_action has failed with status $?" 2>&1 +wait $PID_6 || echo "select_insert_action has failed with status $?" 2>&1 +wait $PID_7 || echo "second select_action has failed with status $?" 2>&1 +wait $PID_8 || echo "second select_insert_action has failed with status $?" 2>&1 + +wait_for_queries_to_finish $WAIT_FINISH $CLICKHOUSE_CLIENT --multiquery --query " -BEGIN TRANSACTION; -SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM src; -SELECT count(), sum(nm) FROM mv"; + BEGIN TRANSACTION; + SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; + COMMIT; +" -$CLICKHOUSE_CLIENT --query "SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM src" -$CLICKHOUSE_CLIENT --query "SELECT count(), sum(nm) FROM mv" +$CLICKHOUSE_CLIENT --multiquery --query " + SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; +" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; $CLICKHOUSE_CLIENT --query "DROP TABLE dst"; 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 143/321] 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 f990f235da10b72e1625d007563761da52067753 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Jul 2024 17:20:19 +0200 Subject: [PATCH 144/321] 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 145/321] bump From 8d4b919bf4d02c09399296e5213525f78bd68a21 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 24 Jul 2024 17:45:40 +0200 Subject: [PATCH 146/321] 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 c3620391b0befaf30eea0eab9001cc98fd5eeecc Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 24 Jul 2024 17:48:07 +0200 Subject: [PATCH 147/321] fix style --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index f6850864be5..718017bca3d 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -41,12 +41,13 @@ function run_until_deadline_and_at_least_times() local min_iterations=$1; shift local function_to_run=$1; shift - local started_time=$(get_now) + local started_time + started_time=$(get_now) local i=0 while true do - $function_to_run $i $@ + $function_to_run $i "$@" [[ $(get_now) -lt $deadline ]] || break 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 148/321] 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 149/321] 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 150/321] 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 151/321] 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 689e31b47e1c85f1ae9721b3928de658eaf9a6ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 18:31:40 +0200 Subject: [PATCH 152/321] More tests --- ...03210_inconsistent_formatting_of_data_types.reference | 6 ++++++ .../03210_inconsistent_formatting_of_data_types.sh | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference index ccb445a0573..836b526905a 100644 --- a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference @@ -1 +1,7 @@ ALTER TABLE columns_with_multiple_streams MODIFY COLUMN `field1` Nullable(tupleElement(x, 2), UInt8) +ALTER TABLE t_update_empty_nested ADD COLUMN `nested.arr2` Array(tuple('- ON NULL -', toLowCardinality(11), 11, 11, toLowCardinality(11), 11), UInt64) +ALTER TABLE t ADD COLUMN `x` Array(tuple(1), UInt8) +ALTER TABLE enum_alter_issue MODIFY COLUMN `a` Enum8(equals('one', timeSlots(timeSlots(arrayEnumerateDense(tuple('0.2147483646', toLowCardinality(toUInt128)), NULL), 4, 12.34, materialize(73), 2)), 1)) +ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN `s` Tuple(Nullable(tupleElement(s, 1), UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String)) +Syntax error +Syntax error diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh index 6cb2d083d71..86c7a5469ca 100755 --- a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh @@ -4,4 +4,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# Ensure that these (possibly incorrect) queries can at least be parsed back after formatting. $CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field1 Nullable(tupleElement(x, 2), UInt8)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t_update_empty_nested ADD COLUMN \`nested.arr2\` Array(tuple('- ON NULL -', toLowCardinality(11), 11, 11, toLowCardinality(11), 11), UInt64)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t ADD COLUMN x Array((1), UInt8)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE enum_alter_issue (MODIFY COLUMN a Enum8(equals('one', timeSlots(timeSlots(arrayEnumerateDense(tuple('0.2147483646', toLowCardinality(toUInt128(12))), NULL), 4, 12.34, materialize(73), 2)), 1)))" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN s Tuple(Nullable(tupleElement(s, 1), UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String))" | $CLICKHOUSE_FORMAT --oneline + +# These invalid queries don't parse and this is normal. +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC((2 + ignore(1, toUInt128(materialize(2)), 2 + toNullable(toNullable(3))), 3), NONE)" 2>&1 | grep -o -F 'Syntax error' +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE test_table ADD COLUMN \`array\` Array(('110', 3, toLowCardinality(3), 3, toNullable(3), toLowCardinality(toNullable(3)), 3), UInt8) DEFAULT [1, 2, 3]" 2>&1 | grep -o -F 'Syntax error' From 5d88f6fc8c4b8fcce4e7a5da073f5d11a86cd3cb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 24 Jul 2024 19:32:20 +0200 Subject: [PATCH 153/321] fix MIN_ITERATIONS --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 718017bca3d..d79ab27d8b2 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -168,7 +168,7 @@ fi START_TIME=$(get_now) STOP_TIME=$((START_TIME + MAIN_TIME_PART)) SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) -MIN_ITERATIONS=50 +MIN_ITERATIONS=30 run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 1 & PID_1=$! run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 2 & PID_2=$! From 67567fcff481793cf7828808c094221ad2ec7389 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 19:51:22 +0200 Subject: [PATCH 154/321] 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 155/321] 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 687c99e39a3ece073239517ffbcecf4612721995 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 24 Jul 2024 18:37:25 +0000 Subject: [PATCH 156/321] try to fix --- .../0_stateless/02680_mysql_ast_logical_err.sql | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql index bde91df83ca..78ce1b68b0d 100644 --- a/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql +++ b/tests/queries/0_stateless/02680_mysql_ast_logical_err.sql @@ -1,4 +1,10 @@ CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; -SELECT count() FROM mysql(mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', ''), '127.0.0.1:9004', currentDatabase(), 'foo', '', ''); -- { serverError UNKNOWN_FUNCTION } -SELECT count() FROM mysql(mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', '', SETTINGS connection_pool_size = 1), '127.0.0.1:9004', currentDatabase(), 'foo', '', ''); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } +SELECT count() FROM mysql( + mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', ''), + '127.0.0.1:9004', currentDatabase(), 'foo', '', '', + SETTINGS connect_timeout = 100, connection_wait_timeout = 100, read_write_timeout = 300); -- { serverError UNKNOWN_FUNCTION } +SELECT count() FROM mysql( + mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', '', SETTINGS connection_pool_size = 1), + '127.0.0.1:9004', currentDatabase(), 'foo', '', '', + SETTINGS connect_timeout = 100, connection_wait_timeout = 100, read_write_timeout = 300); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } From a6a9b8c27204f96e373c9625145dc1609cb7ca8f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 25 Jul 2024 00:49:28 +0200 Subject: [PATCH 157/321] Fix flaky 02447_drop_replica test --- tests/queries/0_stateless/02447_drop_database_replica.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index 93a5fcee8e2..c6bf298f944 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -1,5 +1,9 @@ #!/usr/bin/env bash +# Tags: no-parallel +# no-parallel: This test is not parallel because when we execute system-wide SYSTEM DROP REPLICA, +# other tests might shut down the storage in parallel and the test will fail. + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 8df648b3c8bbc22cee9657145b825e9d991e3c8e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 00:56:41 +0200 Subject: [PATCH 158/321] fix a test, add retries for sql tests --- src/Client/ClientBase.cpp | 12 ++++- src/Client/TestHint.cpp | 52 ++++++++++++++++++- src/Client/TestHint.h | 6 +++ .../02446_parent_zero_copy_locks.sql | 14 +++-- 4 files changed, 76 insertions(+), 8 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13dce05cabc..149e1899ac3 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2230,6 +2230,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) ASTPtr parsed_query; std::unique_ptr current_exception; + size_t retries_count = 0; + while (true) { auto stage = analyzeMultiQueryText(this_query_begin, this_query_end, all_queries_end, @@ -2310,7 +2312,12 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // Check whether the error (or its absence) matches the test hints // (or their absence). bool error_matches_hint = true; - if (have_error) + bool need_retry = test_hint.needRetry(server_exception, &retries_count); + if (need_retry) + { + std::this_thread::sleep_for(std::chrono::seconds(1)); + } + else if (have_error) { if (test_hint.hasServerErrors()) { @@ -2404,7 +2411,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) if (have_error && !ignore_error) return is_interactive; - this_query_begin = this_query_end; + if (!need_retry) + this_query_begin = this_query_end; break; } } diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index b64882577ee..74c65009a73 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -10,6 +10,7 @@ namespace DB::ErrorCodes { extern const int CANNOT_PARSE_TEXT; + extern const int OK; } namespace DB @@ -62,9 +63,28 @@ bool TestHint::hasExpectedServerError(int error) return std::find(server_errors.begin(), server_errors.end(), error) != server_errors.end(); } +bool TestHint::needRetry(const std::unique_ptr & server_exception, size_t * retries_counter) +{ + chassert(retries_counter); + if (max_retries <= *retries_counter) + return false; + + ++*retries_counter; + + int error = ErrorCodes::OK; + if (server_exception) + error = server_exception->code(); + + + if (retry_until) + return !hasExpectedServerError(error); /// retry until we get the expected error + else + return hasExpectedServerError(error); /// retry while we have the expected error +} + void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) { - std::unordered_set commands{"echo", "echoOn", "echoOff"}; + std::unordered_set commands{"echo", "echoOn", "echoOff", "retry"}; std::unordered_set command_errors{ "serverError", @@ -73,6 +93,9 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) for (Token token = comment_lexer.nextToken(); !token.isEnd(); token = comment_lexer.nextToken()) { + if (token.type == TokenType::Whitespace) + continue; + String item = String(token.begin, token.end); if (token.type == TokenType::BareWord && commands.contains(item)) { @@ -82,6 +105,30 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) echo.emplace(true); if (item == "echoOff") echo.emplace(false); + + if (item == "retry") + { + token = comment_lexer.nextToken(); + while (token.type == TokenType::Whitespace) + token = comment_lexer.nextToken(); + + if (token.type != TokenType::Number) + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Could not parse the number of retries: {}", + std::string_view(token.begin, token.end)); + + max_retries = std::stoul(std::string(token.begin, token.end)); + + token = comment_lexer.nextToken(); + while (token.type == TokenType::Whitespace) + token = comment_lexer.nextToken(); + + if (token.type != TokenType::BareWord || + (std::string_view(token.begin, token.end) != "until" && + std::string_view(token.begin, token.end) != "while")) + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected 'until' or 'while' after the number of retries, got: {}", + std::string_view(token.begin, token.end)); + retry_until = std::string_view(token.begin, token.end) == "until"; + } } else if (!is_leading_hint && token.type == TokenType::BareWord && command_errors.contains(item)) { @@ -133,6 +180,9 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) break; } } + + if (max_retries && server_errors.size() != 1) + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected one serverError after the 'retry N while|until' command"); } } diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index b76c4245df4..bbe7873c08b 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB @@ -65,12 +66,17 @@ public: bool hasExpectedClientError(int error); bool hasExpectedServerError(int error); + bool needRetry(const std::unique_ptr & server_exception, size_t * retries_counter); + private: const String & query; ErrorVector server_errors{}; ErrorVector client_errors{}; std::optional echo; + size_t max_retries = 0; + bool retry_until = false; + void parse(Lexer & comment_lexer, bool is_leading_hint); bool allErrorsExpected(int actual_server_error, int actual_client_error) const diff --git a/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql index 86eda526c72..1cae8ae0237 100644 --- a/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql +++ b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql @@ -7,7 +7,7 @@ create table rmt2 (n int, m int, k int) engine=ReplicatedMergeTree('/test/02446/ settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, old_parts_lifetime=0, cleanup_delay_period=0, max_cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0; -- FIXME zero-copy locks may remain in ZooKeeper forever if we failed to insert a part. --- Probably that's why we have to replace repsistent lock with ephemeral sometimes. +-- Probably that's why we have to replace persistent lock with ephemeral sometimes. -- See also "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss" -- in StorageReplicatedMergeTree::createZeroCopyLockNode set insert_keeper_fault_injection_probability=0; @@ -23,6 +23,10 @@ select sleepEachRow(0.5) as test_does_not_rely_on_this; insert into rmt1 values(5, 5, 5); alter table rmt2 update m = m * 10 where 1 settings mutations_sync=2; +-- wait for parts to be merged +select throwIf(name = 'all_0_5_1_6') from system.parts where database=currentDatabase() and table like 'rmt%' and active +format Null; -- { retry 30 until serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + system sync replica rmt2; set optimize_throw_if_noop=1; optimize table rmt2 final; @@ -32,10 +36,10 @@ select 1, * from rmt1 order by n; system sync replica rmt1; select 2, * from rmt2 order by n; --- a funny way to wait for outdated parts to be removed -select sleep(1), sleepEachRow(0.1) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( - 'select *, _state from system.parts where database=''' || currentDatabase() || ''' and table like ''rmt%'' and active=0' - ), 'LineAsString', 's String') settings max_threads=1 format Null; +-- wait for outdated parts to be removed +select throwIf(count() = 0) from ( +select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0 +) format Null; -- { retry 30 until serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0; From fa437b34ec16a0f7fe52f0e4261f964fef2ed606 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jul 2024 03:16:20 +0200 Subject: [PATCH 159/321] 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 160/321] 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 161/321] 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 162/321] 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 163/321] 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 164/321] 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 165/321] 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 166/321] 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 c6a643f981505d0293358c912723f1aece480c7c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 25 Jul 2024 10:31:38 +0200 Subject: [PATCH 167/321] Update tests/queries/0_stateless/02992_all_columns_should_have_comment.sql --- .../0_stateless/02992_all_columns_should_have_comment.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql index 0d34b033354..ad056384bfd 100644 --- a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql +++ b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql @@ -3,6 +3,6 @@ SELECT 'Column ' || name || ' from table ' || concat(database, '.', table) || ' FROM system.columns WHERE (database = 'system') AND (comment = '') AND - (table NOT ILIKE '%\_log\_%') AND + (table NOT ILIKE '%log%') AND (table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log', 'filesystem_read_prefetches_log')) AND (default_kind != 'ALIAS'); From b5171df7798323761b366f01d401c0559ff4c736 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 25 Jul 2024 10:32:52 +0200 Subject: [PATCH 168/321] Update test 03198_table_function_directory_path.sql --- .../0_stateless/03198_table_function_directory_path.reference | 1 + .../queries/0_stateless/03198_table_function_directory_path.sql | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.reference b/tests/queries/0_stateless/03198_table_function_directory_path.reference index 19920de3d3c..74cd8c6d31f 100644 --- a/tests/queries/0_stateless/03198_table_function_directory_path.reference +++ b/tests/queries/0_stateless/03198_table_function_directory_path.reference @@ -1,3 +1,4 @@ 2 2 1 +1 diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.sql b/tests/queries/0_stateless/03198_table_function_directory_path.sql index 9e2791847af..90f687ed6a3 100644 --- a/tests/queries/0_stateless/03198_table_function_directory_path.sql +++ b/tests/queries/0_stateless/03198_table_function_directory_path.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +INSERT INTO FUNCTION file('data_03198_table_function_directory_path.csv', 'csv') SELECT '1.csv' SETTINGS engine_file_truncate_on_insert=1; INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv' SETTINGS engine_file_truncate_on_insert=1; INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv' SETTINGS engine_file_truncate_on_insert=1; INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv' SETTINGS engine_file_truncate_on_insert=1; @@ -11,3 +12,4 @@ SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/'); SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/dir'); SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/*/dir', 'csv'); -- { serverError 74, 636 } SELECT COUNT(*) FROM file('data_03198_table_function_directory_pat'); -- { serverError 400 } +SELECT COUNT(*) FROM file('data_03198_table_function_directory_path.csv'); From 60f529f667069c15fa49296ac1f59a33d94d3f31 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 11:06:00 +0200 Subject: [PATCH 169/321] The most precise way of tracking flushing time in 01246_buffer_flush Right now there are couple of issues with the test: - it does not takes into account INSERT time - it does not takes into account SELECT time, which can be significant from time to time, for instance here [1] it takes 3.3 seconds (and due to tsan build it is not possible to find out why) 2024.07.23 20:52:18.238844 [ 13045 ] {d903650b-ab87-44f3-b7c3-4145e02f1301} executeQuery: (from [::1]:39430) (comment: 01246_buffer_flush.sh) select count() from data_01256; (stage: Complete) 2024.07.23 20:52:21.588183 [ 13045 ] {d903650b-ab87-44f3-b7c3-4145e02f1301} TCPHandler: Processed in 3.354887498 sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/66934/919005c4f70b044ecd9cc1bbce5dc5e276e11929/stateless_tests__tsan__s3_storage__[4_4].html Anyway all of this can be fixed by using QueryStart-insert into data table time. Signed-off-by: Azat Khuzhin --- .../queries/0_stateless/01246_buffer_flush.sh | 49 ++++++++++++------- 1 file changed, 31 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01246_buffer_flush.sh b/tests/queries/0_stateless/01246_buffer_flush.sh index 1ca953c80d9..27c3f01f216 100755 --- a/tests/queries/0_stateless/01246_buffer_flush.sh +++ b/tests/queries/0_stateless/01246_buffer_flush.sh @@ -5,59 +5,72 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -function elapsed_sec() +set -e + +function wait_until() { local expr=$1 && shift - local start end - start=$(date +%s.%N) while ! eval "$expr"; do sleep 0.5 done - end=$(date +%s.%N) - $CLICKHOUSE_LOCAL -q "select floor($end-$start)" +} +function get_buffer_delay() +{ + local buffer_insert_id=$1 && shift + $CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + WITH + (SELECT event_time_microseconds FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryStart' AND query_id = '$buffer_insert_id') AS begin_, + (SELECT max(event_time) FROM data_01256) AS end_ + SELECT dateDiff('seconds', begin_, end_)::UInt64; + " } $CLICKHOUSE_CLIENT -nm -q " drop table if exists data_01256; drop table if exists buffer_01256; - create table data_01256 as system.numbers Engine=Memory(); + create table data_01256 (key UInt64, event_time DateTime(6) MATERIALIZED now64(6)) Engine=Memory(); " echo "min" -$CLICKHOUSE_CLIENT -nm -q " - create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, +$CLICKHOUSE_CLIENT -q " + create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1, 2, 100, /* time */ 4, 100, /* rows */ 1, 1e6 /* bytes */ - ); - insert into buffer_01256 select * from system.numbers limit 5; - select count() from data_01256; + ) " -sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 5 ]]') +min_query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --query_id="$min_query_id" -q "insert into buffer_01256 select * from system.numbers limit 5" +$CLICKHOUSE_CLIENT -q "select count() from data_01256" +wait_until '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 5 ]]' +sec=$(get_buffer_delay "$min_query_id") [[ $sec -ge 2 ]] || echo "Buffer flushed too early, min_time=2, flushed after $sec sec" [[ $sec -lt 100 ]] || echo "Buffer flushed too late, max_time=100, flushed after $sec sec" $CLICKHOUSE_CLIENT -q "select count() from data_01256" $CLICKHOUSE_CLIENT -q "drop table buffer_01256" echo "max" -$CLICKHOUSE_CLIENT -nm -q " - create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, +$CLICKHOUSE_CLIENT -q " + create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1, 100, 2, /* time */ 0, 100, /* rows */ 0, 1e6 /* bytes */ ); - insert into buffer_01256 select * from system.numbers limit 5; - select count() from data_01256; " -sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 10 ]]') +max_query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --query_id="$max_query_id" -q "insert into buffer_01256 select * from system.numbers limit 5" +$CLICKHOUSE_CLIENT -q "select count() from data_01256" +wait_until '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 10 ]]' +sec=$(get_buffer_delay "$max_query_id") [[ $sec -ge 2 ]] || echo "Buffer flushed too early, max_time=2, flushed after $sec sec" $CLICKHOUSE_CLIENT -q "select count() from data_01256" $CLICKHOUSE_CLIENT -q "drop table buffer_01256" echo "direct" $CLICKHOUSE_CLIENT -nm -q " - create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1, 100, 100, /* time */ 0, 9, /* rows */ 0, 1e6 /* bytes */ From 6968945373b2a73c135b0025cf892e21a2af4dbf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Jul 2024 09:58:32 +0000 Subject: [PATCH 170/321] Functions [s-t]*: Iterate over input_rows_count where appropriate --- src/Functions/FunctionTokens.h | 9 +- src/Functions/seriesDecomposeSTL.cpp | 4 +- src/Functions/space.cpp | 50 +++---- src/Functions/stem.cpp | 9 +- src/Functions/stringCutToZero.cpp | 21 ++- src/Functions/substringIndex.cpp | 22 ++-- src/Functions/subtractNanoseconds.cpp | 1 + src/Functions/throwIf.cpp | 2 +- src/Functions/timeSlots.cpp | 80 ++++++------ src/Functions/toDecimalString.cpp | 64 ++++----- src/Functions/toStartOfInterval.cpp | 44 +++---- src/Functions/tokenExtractors.cpp | 18 +-- src/Functions/transform.cpp | 167 ++++++++++++------------ src/Functions/translate.cpp | 16 ++- src/Functions/tupleToNameValuePairs.cpp | 6 +- 15 files changed, 251 insertions(+), 262 deletions(-) diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h index f1435ca5651..b6d8e9ee589 100644 --- a/src/Functions/FunctionTokens.h +++ b/src/Functions/FunctionTokens.h @@ -84,7 +84,7 @@ public: return std::make_shared(std::make_shared()); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { Generator generator; generator.init(arguments, max_substrings_includes_remaining_string); @@ -107,18 +107,17 @@ public: const ColumnString::Chars & src_chars = col_str->getChars(); const ColumnString::Offsets & src_offsets = col_str->getOffsets(); - res_offsets.reserve(src_offsets.size()); - res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random. + res_offsets.reserve(input_rows_count); + res_strings_offsets.reserve(input_rows_count * 5); /// Constant 5 - at random. res_strings_chars.reserve(src_chars.size()); Pos token_begin = nullptr; Pos token_end = nullptr; - size_t size = src_offsets.size(); ColumnString::Offset current_src_offset = 0; ColumnArray::Offset current_dst_offset = 0; ColumnString::Offset current_dst_strings_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { Pos pos = reinterpret_cast(&src_chars[current_src_offset]); current_src_offset = src_offsets[i]; diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 720aa1e0799..1e1c41cafad 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -50,7 +50,7 @@ public: return std::make_shared(std::make_shared(std::make_shared())); } - 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 { ColumnPtr array_ptr = arguments[0].column; const ColumnArray * array = checkAndGetColumn(array_ptr.get()); @@ -79,7 +79,7 @@ public: ColumnArray::Offset prev_src_offset = 0; - for (size_t i = 0; i < src_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { UInt64 period; auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index cd6ca73c088..cf1634e0319 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -55,7 +55,7 @@ public: template - bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars, size_t input_rows_count) const { const ColumnConst & col_times_const = checkAndGetColumn(*col_times); @@ -71,12 +71,12 @@ public: checkRepeatTime(times); - res_offsets.resize(col_times->size()); - res_chars.resize(col_times->size() * (times + 1)); + res_offsets.resize(input_rows_count); + res_chars.resize(input_rows_count * (times + 1)); size_t pos = 0; - for (size_t i = 0; i < col_times->size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { memset(res_chars.begin() + pos, space, times); pos += times; @@ -92,20 +92,20 @@ public: template - bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars, size_t input_rows_count) const { auto * col_times = checkAndGetColumn(col_times_.get()); if (!col_times) return false; - res_offsets.resize(col_times->size()); - res_chars.resize(col_times->size() * 10); /// heuristic + res_offsets.resize(input_rows_count); + res_chars.resize(input_rows_count * 10); /// heuristic const PaddedPODArray & times_data = col_times->getData(); size_t pos = 0; - for (size_t i = 0; i < col_times->size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { typename DataType::FieldType times = times_data[i]; @@ -132,7 +132,7 @@ public: } - 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 auto & col_num = arguments[0].column; @@ -143,26 +143,26 @@ public: if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) { - if ((executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars)) - || (executeConstant(col_num, res_offsets, res_chars))) + if ((executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count)) + || (executeConstant(col_num, res_offsets, res_chars, input_rows_count))) return col_res; } else { - if ((executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars)) - || (executeVector(col_num, res_offsets, res_chars))) + if ((executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count)) + || (executeVector(col_num, res_offsets, res_chars, input_rows_count))) return col_res; } diff --git a/src/Functions/stem.cpp b/src/Functions/stem.cpp index 5b845cf332b..b3be40f4022 100644 --- a/src/Functions/stem.cpp +++ b/src/Functions/stem.cpp @@ -32,7 +32,8 @@ struct StemImpl const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, - const String & language) + const String & language, + size_t input_rows_count) { sb_stemmer * stemmer = sb_stemmer_new(language.data(), "UTF_8"); @@ -45,7 +46,7 @@ struct StemImpl res_offsets.assign(offsets); UInt64 data_size = 0; - for (UInt64 i = 0; i < offsets.size(); ++i) + for (UInt64 i = 0; i < input_rows_count; ++i) { /// Note that accessing -1th element is valid for PaddedPODArray. size_t original_size = offsets[i] - offsets[i - 1]; @@ -101,7 +102,7 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } - 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 auto & langcolumn = arguments[0].column; const auto & strcolumn = arguments[1].column; @@ -119,7 +120,7 @@ public: String language = lang_col->getValue(); auto col_res = ColumnString::create(); - StemImpl::vector(words_col->getChars(), words_col->getOffsets(), col_res->getChars(), col_res->getOffsets(), language); + StemImpl::vector(words_col->getChars(), words_col->getOffsets(), col_res->getChars(), col_res->getOffsets(), language, input_rows_count); return col_res; } }; diff --git a/src/Functions/stringCutToZero.cpp b/src/Functions/stringCutToZero.cpp index b9f742cd8bc..16e57d741fa 100644 --- a/src/Functions/stringCutToZero.cpp +++ b/src/Functions/stringCutToZero.cpp @@ -40,7 +40,7 @@ public: bool useDefaultImplementationForConstants() const override { return true; } - static bool tryExecuteString(const IColumn * col, ColumnPtr & col_res) + static bool tryExecuteString(const IColumn * col, ColumnPtr & col_res, size_t input_rows_count) { const ColumnString * col_str_in = checkAndGetColumn(col); @@ -53,8 +53,7 @@ public: const ColumnString::Chars & in_vec = col_str_in->getChars(); const ColumnString::Offsets & in_offsets = col_str_in->getOffsets(); - size_t size = in_offsets.size(); - out_offsets.resize(size); + out_offsets.resize(input_rows_count); out_vec.resize(in_vec.size()); char * begin = reinterpret_cast(out_vec.data()); @@ -62,7 +61,7 @@ public: ColumnString::Offset current_in_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char * pos_in = reinterpret_cast(&in_vec[current_in_offset]); size_t current_size = strlen(pos_in); @@ -87,7 +86,7 @@ public: } } - static bool tryExecuteFixedString(const IColumn * col, ColumnPtr & col_res) + static bool tryExecuteFixedString(const IColumn * col, ColumnPtr & col_res, size_t input_rows_count) { const ColumnFixedString * col_fstr_in = checkAndGetColumn(col); @@ -99,10 +98,8 @@ public: const ColumnString::Chars & in_vec = col_fstr_in->getChars(); - size_t size = col_fstr_in->size(); - - out_offsets.resize(size); - out_vec.resize(in_vec.size() + size); + out_offsets.resize(input_rows_count); + out_vec.resize(in_vec.size() + input_rows_count); char * begin = reinterpret_cast(out_vec.data()); char * pos = begin; @@ -110,7 +107,7 @@ public: size_t n = col_fstr_in->getN(); - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t current_size = strnlen(pos_in, n); memcpySmallAllowReadWriteOverflow15(pos, pos_in, current_size); @@ -133,12 +130,12 @@ public: } } - 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 IColumn * column = arguments[0].column.get(); ColumnPtr res_column; - if (tryExecuteFixedString(column, res_column) || tryExecuteString(column, res_column)) + if (tryExecuteFixedString(column, res_column, input_rows_count) || tryExecuteString(column, res_column, input_rows_count)) return res_column; throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index eccd849059b..dc12ae193ff 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -68,7 +68,7 @@ namespace return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { ColumnPtr column_string = arguments[0].column; ColumnPtr column_delim = arguments[1].column; @@ -110,10 +110,10 @@ namespace if (is_count_const) { Int64 count = column_count->getInt(0); - vectorConstant(col_str, delim, count, vec_res, offsets_res); + vectorConstant(col_str, delim, count, vec_res, offsets_res, input_rows_count); } else - vectorVector(col_str, delim, column_count.get(), vec_res, offsets_res); + vectorVector(col_str, delim, column_count.get(), vec_res, offsets_res, input_rows_count); } return column_res; } @@ -124,18 +124,18 @@ namespace const String & delim, const IColumn * count_column, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - size_t rows = str_column->size(); res_data.reserve(str_column->getChars().size() / 2); - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) && isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { StringRef str_ref = str_column->getDataAt(i); Int64 count = count_column->getInt(i); @@ -157,18 +157,18 @@ namespace const String & delim, Int64 count, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { - size_t rows = str_column->size(); res_data.reserve(str_column->getChars().size() / 2); - res_offsets.reserve(rows); + res_offsets.reserve(input_rows_count); bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) && isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { StringRef str_ref = str_column->getDataAt(i); diff --git a/src/Functions/subtractNanoseconds.cpp b/src/Functions/subtractNanoseconds.cpp index fffb4eae37a..360c5ecd9cb 100644 --- a/src/Functions/subtractNanoseconds.cpp +++ b/src/Functions/subtractNanoseconds.cpp @@ -6,6 +6,7 @@ namespace DB { using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval; + REGISTER_FUNCTION(SubtractNanoseconds) { factory.registerFunction(); diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index becc6d2f772..e317c65c622 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -152,7 +152,7 @@ private: return nullptr; } - bool allow_custom_error_code_argument; + const bool allow_custom_error_code_argument; }; } diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 040495ab023..b62bb20c64e 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -41,18 +41,17 @@ struct TimeSlotsImpl /// The following three methods process DateTime type static void vectorVector( const PaddedPODArray & starts, const PaddedPODArray & durations, UInt32 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, + size_t input_rows_count) { if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - size_t size = starts.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + durations[i]) / time_slot_size; value <= end; ++value) { @@ -66,18 +65,17 @@ struct TimeSlotsImpl static void vectorConstant( const PaddedPODArray & starts, UInt32 duration, UInt32 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, + size_t input_rows_count) { if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - size_t size = starts.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; ++value) { @@ -91,18 +89,17 @@ struct TimeSlotsImpl static void constantVector( UInt32 start, const PaddedPODArray & durations, UInt32 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, + size_t input_rows_count) { if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - size_t size = durations.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (UInt32 value = start / time_slot_size, end = (start + durations[i]) / time_slot_size; value <= end; ++value) { @@ -120,12 +117,11 @@ struct TimeSlotsImpl */ static NO_SANITIZE_UNDEFINED void vectorVector( const PaddedPODArray & starts, const PaddedPODArray & durations, Decimal64 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale, + size_t input_rows_count) { - size_t size = starts.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); /// Modify all units to have same scale UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); @@ -139,7 +135,7 @@ struct TimeSlotsImpl if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1) { @@ -152,12 +148,11 @@ struct TimeSlotsImpl static NO_SANITIZE_UNDEFINED void vectorConstant( const PaddedPODArray & starts, Decimal64 duration, Decimal64 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale, + size_t input_rows_count) { - size_t size = starts.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); /// Modify all units to have same scale UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); @@ -172,7 +167,7 @@ struct TimeSlotsImpl if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + duration) / time_slot_size; value <= end; value += 1) { @@ -185,12 +180,11 @@ struct TimeSlotsImpl static NO_SANITIZE_UNDEFINED void constantVector( DateTime64 start, const PaddedPODArray & durations, Decimal64 time_slot_size, - PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) + PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale, + size_t input_rows_count) { - size_t size = durations.size(); - - result_offsets.resize(size); - result_values.reserve(size); + result_offsets.resize(input_rows_count); + result_values.reserve(input_rows_count); /// Modify all units to have same scale UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); @@ -205,7 +199,7 @@ struct TimeSlotsImpl if (time_slot_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { for (DateTime64 value = start / time_slot_size, end = (start + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1) { @@ -282,7 +276,7 @@ public: } - 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 { if (WhichDataType(arguments[0].type).isDateTime()) { @@ -308,17 +302,17 @@ public: if (dt_starts && durations) { - TimeSlotsImpl::vectorVector(dt_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets()); + TimeSlotsImpl::vectorVector(dt_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), input_rows_count); return res; } else if (dt_starts && const_durations) { - TimeSlotsImpl::vectorConstant(dt_starts->getData(), const_durations->getValue(), time_slot_size, res_values, res->getOffsets()); + TimeSlotsImpl::vectorConstant(dt_starts->getData(), const_durations->getValue(), time_slot_size, res_values, res->getOffsets(), input_rows_count); return res; } else if (dt_const_starts && durations) { - TimeSlotsImpl::constantVector(dt_const_starts->getValue(), durations->getData(), time_slot_size, res_values, res->getOffsets()); + TimeSlotsImpl::constantVector(dt_const_starts->getValue(), durations->getData(), time_slot_size, res_values, res->getOffsets(), input_rows_count); return res; } } @@ -353,21 +347,21 @@ public: if (starts && durations) { TimeSlotsImpl::vectorVector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), - start_time_scale, duration_scale, time_slot_scale); + start_time_scale, duration_scale, time_slot_scale, input_rows_count); return res; } else if (starts && const_durations) { TimeSlotsImpl::vectorConstant( starts->getData(), const_durations->getValue(), time_slot_size, res_values, res->getOffsets(), - start_time_scale, duration_scale, time_slot_scale); + start_time_scale, duration_scale, time_slot_scale, input_rows_count); return res; } else if (const_starts && durations) { TimeSlotsImpl::constantVector( const_starts->getValue(), durations->getData(), time_slot_size, res_values, res->getOffsets(), - start_time_scale, duration_scale, time_slot_scale); + start_time_scale, duration_scale, time_slot_scale, input_rows_count); return res; } } diff --git a/src/Functions/toDecimalString.cpp b/src/Functions/toDecimalString.cpp index 523948a5396..3566ebc93ad 100644 --- a/src/Functions/toDecimalString.cpp +++ b/src/Functions/toDecimalString.cpp @@ -54,9 +54,9 @@ private: /// For operations with Integer/Float template void vectorConstant(const FromVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, + size_t input_rows_count) const { - size_t input_rows_count = vec_from.size(); result_offsets.resize(input_rows_count); /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, @@ -74,9 +74,9 @@ private: template void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, + size_t input_rows_count) const { - size_t input_rows_count = vec_from.size(); result_offsets.resize(input_rows_count); WriteBufferFromVector buf_to(vec_to); @@ -98,7 +98,8 @@ private: /// For operations with Decimal template void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale, + size_t input_rows_count) const { /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. constexpr size_t max_digits = std::numeric_limits::digits10; @@ -107,7 +108,6 @@ private: "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); WriteBufferFromVector buf_to(vec_to); - size_t input_rows_count = vec_from.size(); result_offsets.resize(input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) @@ -121,9 +121,9 @@ private: template void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale, + size_t input_rows_count) const { - size_t input_rows_count = vec_from.size(); result_offsets.resize(input_rows_count); WriteBufferFromVector buf_to(vec_to); @@ -182,28 +182,28 @@ private: } public: - 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 { switch (arguments[0].type->getTypeId()) { - case TypeIndex::UInt8: return executeType(arguments); - case TypeIndex::UInt16: return executeType(arguments); - case TypeIndex::UInt32: return executeType(arguments); - case TypeIndex::UInt64: return executeType(arguments); - case TypeIndex::UInt128: return executeType(arguments); - case TypeIndex::UInt256: return executeType(arguments); - case TypeIndex::Int8: return executeType(arguments); - case TypeIndex::Int16: return executeType(arguments); - case TypeIndex::Int32: return executeType(arguments); - case TypeIndex::Int64: return executeType(arguments); - case TypeIndex::Int128: return executeType(arguments); - case TypeIndex::Int256: return executeType(arguments); - case TypeIndex::Float32: return executeType(arguments); - case TypeIndex::Float64: return executeType(arguments); - case TypeIndex::Decimal32: return executeType(arguments); - case TypeIndex::Decimal64: return executeType(arguments); - case TypeIndex::Decimal128: return executeType(arguments); - case TypeIndex::Decimal256: return executeType(arguments); + case TypeIndex::UInt8: return executeType(arguments, input_rows_count); + case TypeIndex::UInt16: return executeType(arguments, input_rows_count); + case TypeIndex::UInt32: return executeType(arguments, input_rows_count); + case TypeIndex::UInt64: return executeType(arguments, input_rows_count); + case TypeIndex::UInt128: return executeType(arguments, input_rows_count); + case TypeIndex::UInt256: return executeType(arguments, input_rows_count); + case TypeIndex::Int8: return executeType(arguments, input_rows_count); + case TypeIndex::Int16: return executeType(arguments, input_rows_count); + case TypeIndex::Int32: return executeType(arguments, input_rows_count); + case TypeIndex::Int64: return executeType(arguments, input_rows_count); + case TypeIndex::Int128: return executeType(arguments, input_rows_count); + case TypeIndex::Int256: return executeType(arguments, input_rows_count); + case TypeIndex::Float32: return executeType(arguments, input_rows_count); + case TypeIndex::Float64: return executeType(arguments, input_rows_count); + case TypeIndex::Decimal32: return executeType(arguments, input_rows_count); + case TypeIndex::Decimal64: return executeType(arguments, input_rows_count); + case TypeIndex::Decimal128: return executeType(arguments, input_rows_count); + case TypeIndex::Decimal256: return executeType(arguments, input_rows_count); default: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); @@ -212,7 +212,7 @@ public: private: template - ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); const auto * precision_col_const = checkAndGetColumnConst>(arguments[1].column.get()); @@ -230,9 +230,9 @@ private: { UInt8 from_scale = from_col->getScale(); if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale, input_rows_count); else if (precision_col) - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale, input_rows_count); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); } @@ -245,9 +245,9 @@ private: if (from_col) { if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, input_rows_count); else if (precision_col) - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, input_rows_count); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 50442d1b448..21b7cf895d2 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -147,19 +147,20 @@ public: std::unreachable(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const auto & time_column = arguments[0]; const auto & interval_column = arguments[1]; const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone); + auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone, input_rows_count); return result_column; } private: ColumnPtr dispatchForTimeColumn( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone) const + const DataTypePtr & result_type, const DateLUTImpl & time_zone, + size_t input_rows_count) const { const auto & time_column_type = *time_column.type.get(); const auto & time_column_col = *time_column.column.get(); @@ -170,19 +171,19 @@ private: auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count, scale); } else if (isDateTime(time_column_type)) { const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } else if (isDate(time_column_type)) { const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } @@ -190,7 +191,7 @@ private: template ColumnPtr dispatchForIntervalColumn( const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const + const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) @@ -207,27 +208,27 @@ private: switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Kind::Nanosecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Microsecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Millisecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Second: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Minute: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Hour: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Day: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Week: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Month: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Quarter: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); case IntervalKind::Kind::Year: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale); } std::unreachable(); @@ -236,22 +237,21 @@ private: template ColumnPtr execute( const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const + const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale) const { using ResultColumnType = typename ResultDataType::ColumnType; using ResultFieldType = typename ResultDataType::FieldType; const auto & time_data = time_column_type.getData(); - size_t size = time_data.size(); auto result_col = result_type->createColumn(); auto * col_to = assert_cast(result_col.get()); auto & result_data = col_to->getData(); - result_data.resize(size); + result_data.resize(input_rows_count); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); - for (size_t i = 0; i != size; ++i) + for (size_t i = 0; i != input_rows_count; ++i) result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); return result_col; diff --git a/src/Functions/tokenExtractors.cpp b/src/Functions/tokenExtractors.cpp index e7dcb5cced3..1bbf313fbae 100644 --- a/src/Functions/tokenExtractors.cpp +++ b/src/Functions/tokenExtractors.cpp @@ -73,7 +73,7 @@ public: return std::make_shared(std::make_shared()); } - 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 { auto column_offsets = ColumnArray::ColumnOffsets::create(); @@ -90,9 +90,9 @@ public: auto input_column = arguments[0].column; if (const auto * column_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_string, *result_column_string, *column_offsets); + executeImpl(extractor, *column_string, *result_column_string, *column_offsets, input_rows_count); else if (const auto * column_fixed_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets); + executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets, input_rows_count); return ColumnArray::create(std::move(result_column_string), std::move(column_offsets)); } @@ -105,9 +105,9 @@ public: auto input_column = arguments[0].column; if (const auto * column_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_string, *result_column_string, *column_offsets); + executeImpl(extractor, *column_string, *result_column_string, *column_offsets, input_rows_count); else if (const auto * column_fixed_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets); + executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets, input_rows_count); return ColumnArray::create(std::move(result_column_string), std::move(column_offsets)); } @@ -120,15 +120,15 @@ private: const ExtractorType & extractor, StringColumnType & input_data_column, ResultStringColumnType & result_data_column, - ColumnArray::ColumnOffsets & offsets_column) const + ColumnArray::ColumnOffsets & offsets_column, + size_t input_rows_count) const { size_t current_tokens_size = 0; auto & offsets_data = offsets_column.getData(); - size_t column_size = input_data_column.size(); - offsets_data.resize(column_size); + offsets_data.resize(input_rows_count); - for (size_t i = 0; i < column_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { auto data = input_data_column.getDataAt(i); diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 68500779f93..0dfc9197845 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -173,30 +173,30 @@ namespace } else if (cache.table_num_to_idx) { - if (!executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted) - && !executeNum>(in, *column_result, default_non_const, *in_casted)) + if (!executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count) + && !executeNum>(in, *column_result, default_non_const, *in_casted, input_rows_count)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); } } else if (cache.table_string_to_idx) { - if (!executeString(in, *column_result, default_non_const, *in_casted)) - executeContiguous(in, *column_result, default_non_const, *in_casted); + if (!executeString(in, *column_result, default_non_const, *in_casted, input_rows_count)) + executeContiguous(in, *column_result, default_non_const, *in_casted, input_rows_count); } else if (cache.table_anything_to_idx) { - executeAnything(in, *column_result, default_non_const, *in_casted); + executeAnything(in, *column_result, default_non_const, *in_casted, input_rows_count); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); @@ -217,12 +217,11 @@ namespace return impl->execute(args, result_type, input_rows_count); } - void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const + void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const { - const size_t size = in->size(); const auto & table = *cache.table_anything_to_idx; - column_result.reserve(size); - for (size_t i = 0; i < size; ++i) + column_result.reserve(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) { SipHash hash; in->updateHashWithValue(i, hash); @@ -239,12 +238,11 @@ namespace } } - void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const + void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const { - const size_t size = in->size(); const auto & table = *cache.table_string_to_idx; - column_result.reserve(size); - for (size_t i = 0; i < size; ++i) + column_result.reserve(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) { const auto * it = table.find(in->getDataAt(i)); if (it) @@ -259,7 +257,7 @@ namespace } template - bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const + bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -269,24 +267,23 @@ namespace if constexpr (std::is_same_v, T> || std::is_same_v, T>) in_scale = in->getScale(); - if (!executeNumToString(pod, column_result, default_non_const) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale) - && !executeNumToNum>(pod, column_result, default_non_const, in_scale)) + if (!executeNumToString(pod, column_result, default_non_const, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count) + && !executeNumToNum>(pod, column_result, default_non_const, in_scale, input_rows_count)) { - const size_t size = pod.size(); const auto & table = *cache.table_num_to_idx; - column_result.reserve(size); - for (size_t i = 0; i < size; ++i) + column_result.reserve(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) { const auto * it = table.find(bit_cast(pod[i])); if (it) @@ -303,14 +300,13 @@ namespace } template - bool executeNumToString(const PaddedPODArray & pod, IColumn & column_result, const ColumnPtr default_non_const) const + bool executeNumToString(const PaddedPODArray & pod, IColumn & column_result, const ColumnPtr default_non_const, size_t input_rows_count) const { auto * out = typeid_cast(&column_result); if (!out) return false; auto & out_offs = out->getOffsets(); - const size_t size = pod.size(); - out_offs.resize(size); + out_offs.resize(input_rows_count); auto & out_chars = out->getChars(); const auto * to_col = assert_cast(cache.to_column.get()); @@ -325,14 +321,14 @@ namespace const auto & def_offs = def->getOffsets(); const auto * def_data = def_chars.data(); auto def_size = def_offs[0]; - executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_data, def_size, size); + executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_data, def_size, input_rows_count); } else { const auto * def = assert_cast(default_non_const.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); - executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); + executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, input_rows_count); } return true; } @@ -347,10 +343,10 @@ namespace const ColumnString::Offsets & to_offsets, const DefData & def_data, const DefOffs & def_offsets, - const size_t size) const + size_t input_rows_count) const { size_t out_cur_off = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char8_t * to = nullptr; size_t to_size = 0; @@ -382,14 +378,13 @@ namespace template bool executeNumToNum( - const PaddedPODArray & pod, IColumn & column_result, const ColumnPtr default_non_const, const UInt32 in_scale) const + const PaddedPODArray & pod, IColumn & column_result, ColumnPtr default_non_const, UInt32 in_scale, size_t input_rows_count) const { auto * out = typeid_cast(&column_result); if (!out) return false; auto & out_pod = out->getData(); - const size_t size = pod.size(); - out_pod.resize(size); + out_pod.resize(input_rows_count); UInt32 out_scale = 0; if constexpr (std::is_same_v, T> || std::is_same_v, T>) out_scale = out->getScale(); @@ -399,15 +394,15 @@ namespace if (cache.default_column) { const auto const_def = assert_cast(cache.default_column.get())->getData()[0]; - executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, size, out_scale, out_scale); + executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, input_rows_count, out_scale, out_scale); } else if (default_non_const) { const auto & nconst_def = assert_cast(default_non_const.get())->getData(); - executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, size, out_scale, out_scale); + executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, input_rows_count, out_scale, out_scale); } else - executeNumToNumHelper(table, pod, out_pod, to_pod, pod, size, out_scale, in_scale); + executeNumToNumHelper(table, pod, out_pod, to_pod, pod, input_rows_count, out_scale, in_scale); return true; } @@ -418,11 +413,11 @@ namespace PaddedPODArray & out_pod, const PaddedPODArray & to_pod, const Def & def, - const size_t size, - const UInt32 out_scale, - const UInt32 def_scale) const + size_t input_rows_count, + UInt32 out_scale, + UInt32 def_scale) const { - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * it = table.find(bit_cast(pod[i])); if (it) @@ -450,7 +445,7 @@ namespace } } - bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const + bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -458,19 +453,19 @@ namespace const auto & data = in->getChars(); const auto & offsets = in->getOffsets(); - if (!executeStringToString(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const) - && !executeStringToNum>(data, offsets, column_result, default_non_const)) + if (!executeStringToString(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count) + && !executeStringToNum>(data, offsets, column_result, default_non_const, input_rows_count)) { const size_t size = offsets.size(); const auto & table = *cache.table_string_to_idx; @@ -497,14 +492,14 @@ namespace const ColumnString::Chars & data, const ColumnString::Offsets & offsets, IColumn & column_result, - const ColumnPtr default_non_const) const + const ColumnPtr default_non_const, + size_t input_rows_count) const { auto * out = typeid_cast(&column_result); if (!out) return false; auto & out_offs = out->getOffsets(); - const size_t size = offsets.size(); - out_offs.resize(size); + out_offs.resize(input_rows_count); auto & out_chars = out->getChars(); const auto * to_col = assert_cast(cache.to_column.get()); @@ -519,18 +514,18 @@ namespace const auto & def_offs = def->getOffsets(); const auto * def_data = def_chars.data(); auto def_size = def_offs[0]; - executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_data, def_size, size); + executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_data, def_size, input_rows_count); } else if (default_non_const) { const auto * def = assert_cast(default_non_const.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); - executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); + executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, input_rows_count); } else { - executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, data, offsets, size); + executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, data, offsets, input_rows_count); } return true; } @@ -546,11 +541,11 @@ namespace const ColumnString::Offsets & to_offsets, const DefData & def_data, const DefOffs & def_offsets, - const size_t size) const + size_t input_rows_count) const { ColumnString::Offset current_offset = 0; size_t out_cur_off = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const char8_t * to = nullptr; size_t to_size = 0; @@ -587,26 +582,26 @@ namespace const ColumnString::Chars & data, const ColumnString::Offsets & offsets, IColumn & column_result, - const ColumnPtr default_non_const) const + const ColumnPtr default_non_const, + size_t input_rows_count) const { auto * out = typeid_cast(&column_result); if (!out) return false; auto & out_pod = out->getData(); - const size_t size = offsets.size(); - out_pod.resize(size); + out_pod.resize(input_rows_count); const auto & to_pod = assert_cast(cache.to_column.get())->getData(); const auto & table = *cache.table_string_to_idx; if (cache.default_column) { const auto const_def = assert_cast(cache.default_column.get())->getData()[0]; - executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, size); + executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, input_rows_count); } else { const auto & nconst_def = assert_cast(default_non_const.get())->getData(); - executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, size); + executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, input_rows_count); } return true; } @@ -619,10 +614,10 @@ namespace PaddedPODArray & out_pod, const PaddedPODArray & to_pod, const Def & def, - const size_t size) const + size_t input_rows_count) const { ColumnString::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1}; current_offset = offsets[i]; diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index 2df08a5664e..366640d7d20 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -52,7 +52,8 @@ struct TranslateImpl const std::string & map_from, const std::string & map_to, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { Map map; fillMapWithValues(map, map_from, map_to); @@ -62,7 +63,7 @@ struct TranslateImpl UInt8 * dst = res_data.data(); - for (UInt64 i = 0; i < offsets.size(); ++i) + for (UInt64 i = 0; i < input_rows_count; ++i) { const UInt8 * src = data.data() + offsets[i - 1]; const UInt8 * src_end = data.data() + offsets[i] - 1; @@ -175,19 +176,20 @@ struct TranslateUTF8Impl const std::string & map_from, const std::string & map_to, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { MapASCII map_ascii; MapUTF8 map; fillMapWithValues(map_ascii, map, map_from, map_to); res_data.resize(data.size()); - res_offsets.resize(offsets.size()); + res_offsets.resize(input_rows_count); UInt8 * dst = res_data.data(); UInt64 data_size = 0; - for (UInt64 i = 0; i < offsets.size(); ++i) + for (UInt64 i = 0; i < input_rows_count; ++i) { const UInt8 * src = data.data() + offsets[i - 1]; const UInt8 * src_end = data.data() + offsets[i] - 1; @@ -311,7 +313,7 @@ public: } } - 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_src = arguments[0].column; const ColumnPtr column_map_from = arguments[1].column; @@ -330,7 +332,7 @@ public: if (const ColumnString * col = checkAndGetColumn(column_src.get())) { auto col_res = ColumnString::create(); - Impl::vector(col->getChars(), col->getOffsets(), map_from, map_to, col_res->getChars(), col_res->getOffsets()); + Impl::vector(col->getChars(), col->getOffsets(), map_from, map_to, col_res->getChars(), col_res->getOffsets(), input_rows_count); return col_res; } else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_src.get())) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index 998e0da4f0c..92734d3d1fc 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -99,16 +99,16 @@ public: return std::make_shared(item_data_type); } - 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 IColumn * tuple_col = arguments[0].column.get(); const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); - const auto * tuple_col_concrete = assert_cast(tuple_col); + const auto * tuple_col_concrete = assert_cast(tuple_col); auto keys = ColumnString::create(); MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); auto offsets = ColumnVector::create(); - for (size_t row = 0; row < tuple_col_concrete->size(); ++row) + for (size_t row = 0; row < input_rows_count; ++row) { for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col) { From dc2c3fb1ca4653ee006c8cbbbfa32688f19f1992 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 11:57:14 +0200 Subject: [PATCH 171/321] Revert "Merge pull request #66563 from ClickHouse/delete-bad-test" This reverts commit d0753c8bb60dacfbd99687906fe4efb7665b20fa, reversing changes made to b8202e19baf7ad171e232a431c8a4c3f1c86e63e. Signed-off-by: Azat Khuzhin --- .../0_stateless/02805_distributed_queries_timeouts.reference | 0 .../queries/0_stateless/02805_distributed_queries_timeouts.sql | 3 +++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.reference create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.sql diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference b/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql new file mode 100644 index 00000000000..0b7337d1255 --- /dev/null +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -0,0 +1,3 @@ +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 1a4730f1f390e468dab2849bd1b2770e0fb2cbe6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 12:03:50 +0200 Subject: [PATCH 172/321] Use Distributed table to avoid extra DESC queries Signed-off-by: Azat Khuzhin --- .../0_stateless/02805_distributed_queries_timeouts.sql | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql index 0b7337d1255..bfa39cd78ee 100644 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -1,3 +1,4 @@ -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; +create table dist as system.one engine=Distributed(test_shard_localhost, system, one); +select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; +select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; +select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 49732f2966cd793e32234068cf0b87cea9e3eed6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 12:06:10 +0200 Subject: [PATCH 173/321] Tune sleep duration/receive_timeout in 02805_distributed_queries_timeouts Signed-off-by: Azat Khuzhin --- .../0_stateless/02805_distributed_queries_timeouts.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql index bfa39cd78ee..f6bccc99977 100644 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -1,4 +1,4 @@ create table dist as system.one engine=Distributed(test_shard_localhost, system, one); -select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; -select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; -select sleep(3) from dist settings prefer_localhost_replica=0, receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; +select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=1 format Null; +select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=1, use_hedged_requests=0 format Null; +select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 4ee409094d4b1b9a8c8f4da5f1cf55d78a43b11f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Jul 2024 12:12:23 +0200 Subject: [PATCH 174/321] 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 86e23b346fbce791794e38ad7ae77d8af964988a Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 25 Jul 2024 12:12:37 +0200 Subject: [PATCH 175/321] rename test stages --- .github/workflows/master.yml | 21 +++++++++++---------- .github/workflows/pull_request.yml | 20 ++++++++++---------- tests/ci/ci_config.py | 6 +++--- tests/ci/ci_definitions.py | 6 +++--- tests/ci/test_ci_config.py | 4 ++-- 5 files changed, 29 insertions(+), 28 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index acd7511d520..2ce1124404f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -93,21 +93,21 @@ jobs: with: stage: Builds_2 data: ${{ needs.RunConfig.outputs.data }} - Tests_2: + Tests_2_ww: needs: [RunConfig, Builds_2] + if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }} + uses: ./.github/workflows/reusable_test_stage.yml + with: + stage: Tests_2_ww + data: ${{ needs.RunConfig.outputs.data }} + Tests_2: + # Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there. + needs: [RunConfig, Builds_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }} uses: ./.github/workflows/reusable_test_stage.yml with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - Tests_3: - # Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there. - needs: [RunConfig, Builds_1] - if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} - uses: ./.github/workflows/reusable_test_stage.yml - with: - stage: Tests_3 - data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# # Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3 @@ -123,7 +123,7 @@ jobs: FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -133,6 +133,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat > "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 34bf51871d2..854dff530e7 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -123,20 +123,20 @@ jobs: stage: Builds_2 data: ${{ needs.RunConfig.outputs.data }} # stage for running non-required checks without being blocked by required checks (Test_1) if corresponding settings is selected - Tests_2: + Tests_2_ww: needs: [RunConfig, Builds_1] + if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }} + uses: ./.github/workflows/reusable_test_stage.yml + with: + stage: Tests_2_ww + data: ${{ needs.RunConfig.outputs.data }} + Tests_2: + needs: [RunConfig, Builds_1, Tests_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }} uses: ./.github/workflows/reusable_test_stage.yml with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - Tests_3: - needs: [RunConfig, Builds_1, Tests_1] - if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} - uses: ./.github/workflows/reusable_test_stage.yml - with: - stage: Tests_3 - data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) @@ -154,7 +154,7 @@ jobs: if: ${{ !cancelled() }} # 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] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -178,7 +178,7 @@ jobs: # FinishCheck: if: ${{ !failure() && !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a9bdb639835..0e295b2339d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -587,10 +587,10 @@ class CI: if job_name in REQUIRED_CHECKS: stage_type = WorkflowStages.TESTS_1 else: - stage_type = WorkflowStages.TESTS_3 + stage_type = WorkflowStages.TESTS_2 assert stage_type, f"BUG [{job_name}]" - if non_blocking_ci and stage_type == WorkflowStages.TESTS_3: - stage_type = WorkflowStages.TESTS_2 + if non_blocking_ci and stage_type == WorkflowStages.TESTS_2: + stage_type = WorkflowStages.TESTS_2_WW return stage_type @classmethod diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 149177ecba5..054b554b8fa 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -67,10 +67,10 @@ class WorkflowStages(metaclass=WithIter): BUILDS_2 = "Builds_2" # all tests required for merge TESTS_1 = "Tests_1" - # not used atm - TESTS_2 = "Tests_2" + # used in woolenwolfdog mode + TESTS_2_WW = "Tests_2_ww" # all tests not required for merge - TESTS_3 = "Tests_3" + TESTS_2 = "Tests_2" class Runners(metaclass=WithIter): diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 4a2bd606d0e..be540413b3c 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -211,7 +211,7 @@ class TestCIConfig(unittest.TestCase): else: self.assertTrue( CI.get_job_ci_stage(job) - in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_3), + in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_2), msg=f"Stage for [{job}] is not correct", ) @@ -242,7 +242,7 @@ class TestCIConfig(unittest.TestCase): else: self.assertTrue( CI.get_job_ci_stage(job, non_blocking_ci=True) - in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_2), + in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_2_WW), msg=f"Stage for [{job}] is not correct", ) From 0dc67aae97d4b964cb0f9c389cbf3ce91cb76fb7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 25 Jul 2024 13:52:30 +0200 Subject: [PATCH 176/321] fix MIN_ITERATIONS 2 --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index d79ab27d8b2..620281ee972 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -168,7 +168,7 @@ fi START_TIME=$(get_now) STOP_TIME=$((START_TIME + MAIN_TIME_PART)) SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) -MIN_ITERATIONS=30 +MIN_ITERATIONS=25 run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 1 & PID_1=$! run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 2 & PID_2=$! From b80305ba981ca1f862084d3316144efcba17466b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 14:31:00 +0200 Subject: [PATCH 177/321] Improve backport script --- tests/ci/cherry_pick.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index a7fc6d02853..623a816148e 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -420,7 +420,8 @@ class Backport: fetch_release_prs = self.gh.get_release_pulls(self._fetch_from) fetch_release_branches = [pr.head.ref for pr in fetch_release_prs] self.labels_to_backport = [ - f"v{branch}-must-backport" for branch in fetch_release_branches + f"v{branch if self._repo_name == "ClickHouse/ClickHouse" else branch.replace('release/','')}-must-backport" + for branch in fetch_release_branches ] logging.info("Fetching from %s", self._fetch_from) @@ -490,17 +491,23 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if ( - any(label in pr_labels for label in self.must_create_backport_labels) - or self._repo_name != self._fetch_from - ): + if any(label in pr_labels for label in self.must_create_backport_labels): branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches ] # type: List[ReleaseBranch] else: branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) + ReleaseBranch( + ( + br + if self._repo_name == "ClickHouse/clickhouse" + else f"release/{br}" + ), + pr, + self.repo, + self.backport_created_label, + ) for br in [ label.split("-", 1)[0][1:] # v21.8-must-backport for label in pr_labels From 2c83a39503255f0b2233b511a63a262cb8749a53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 12:53:16 +0000 Subject: [PATCH 178/321] Fixes --- tests/ci/cherry_pick.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 623a816148e..c2f567e5f15 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -420,7 +420,9 @@ class Backport: fetch_release_prs = self.gh.get_release_pulls(self._fetch_from) fetch_release_branches = [pr.head.ref for pr in fetch_release_prs] self.labels_to_backport = [ - f"v{branch if self._repo_name == "ClickHouse/ClickHouse" else branch.replace('release/','')}-must-backport" + f"v{branch}-must-backport" + if self._repo_name == "ClickHouse/ClickHouse" + else f"v{branch.replace('release/','')}-must-backport" for branch in fetch_release_branches ] @@ -501,7 +503,7 @@ class Backport: ReleaseBranch( ( br - if self._repo_name == "ClickHouse/clickhouse" + if self._repo_name == "ClickHouse/Clickhouse" else f"release/{br}" ), pr, From 92cca8e65dec9f46d5a248c10e748088c9437cb6 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 25 Jul 2024 14:51:53 +0200 Subject: [PATCH 179/321] Fix --- src/AggregateFunctions/SingleValueData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/SingleValueData.cpp b/src/AggregateFunctions/SingleValueData.cpp index a14caf00f73..996e64b22e0 100644 --- a/src/AggregateFunctions/SingleValueData.cpp +++ b/src/AggregateFunctions/SingleValueData.cpp @@ -1191,7 +1191,7 @@ bool SingleValueDataString::isEqualTo(const DB::IColumn & column, size_t row_num bool SingleValueDataString::isEqualTo(const SingleValueDataBase & other) const { auto const & to = assert_cast(other); - return has() && to.getStringRef() == getStringRef(); + return has() && to.has() && to.getStringRef() == getStringRef(); } void SingleValueDataString::set(const IColumn & column, size_t row_num, Arena * arena) From b23ce171c3620568829201d80789f314fc27499a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 13:11:32 +0000 Subject: [PATCH 180/321] My black version said this was ok --- tests/ci/cherry_pick.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index c2f567e5f15..0b2aa9a2d35 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -420,9 +420,11 @@ class Backport: fetch_release_prs = self.gh.get_release_pulls(self._fetch_from) fetch_release_branches = [pr.head.ref for pr in fetch_release_prs] self.labels_to_backport = [ - f"v{branch}-must-backport" - if self._repo_name == "ClickHouse/ClickHouse" - else f"v{branch.replace('release/','')}-must-backport" + ( + f"v{branch}-must-backport" + if self._repo_name == "ClickHouse/ClickHouse" + else f"v{branch.replace('release/','')}-must-backport" + ) for branch in fetch_release_branches ] From c5164fede8665b61c10ec0d7b6873a7cf04aab12 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 13:17:21 +0000 Subject: [PATCH 181/321] Fix some test. --- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6b5b129085d..5972d89bddd 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,7 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); + before_prewhere_sample = prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1976,7 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where->dag.updateHeader(before_where_sample); + before_where_sample = before_where->dag.updateHeader(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); From ae75c99e3fad02a3716c9d520c3a680f4d9d28e4 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 25 Jul 2024 15:17:59 +0200 Subject: [PATCH 182/321] Add a test --- .../0_stateless/03210_fix-single-value-data-assertion.reference | 0 .../0_stateless/03210_fix-single-value-data-assertion.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/03210_fix-single-value-data-assertion.reference create mode 100644 tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql diff --git a/tests/queries/0_stateless/03210_fix-single-value-data-assertion.reference b/tests/queries/0_stateless/03210_fix-single-value-data-assertion.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql b/tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql new file mode 100644 index 00000000000..66e62377d6b --- /dev/null +++ b/tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql @@ -0,0 +1 @@ +SELECT intDiv(number, 2) AS k, count(toFixedString(toFixedString('hello', 5), 5)) IGNORE NULLS, sumArgMax(number, toString(number % 20)), argMax(toString(number), number) FROM (SELECT number FROM system.numbers LIMIT 65537) WHERE toLowCardinality(toLowCardinality(toNullable(21))) GROUP BY k WITH TOTALS ORDER BY k ASC NULLS FIRST LIMIT 255 SETTINGS group_by_overflow_mode = 'any', totals_mode = 'before_having', max_rows_to_group_by = 100000 FORMAT Null From ad44fb1ba4759434ecb4353a7878aea6162f8fef Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:22:19 +0000 Subject: [PATCH 183/321] Increase lock_acquire_timeout_for_background_operations setting in dynamic merges tests --- .../03037_dynamic_merges_1_horizontal_compact_merge_tree.sql | 2 +- .../03037_dynamic_merges_1_horizontal_compact_wide_tree.sql | 2 +- .../03037_dynamic_merges_1_vertical_compact_merge_tree.sql | 5 +++-- .../03037_dynamic_merges_1_vertical_wide_merge_tree.sql | 2 +- .../03037_dynamic_merges_2_horizontal_compact_merge_tree.sql | 2 +- .../03037_dynamic_merges_2_horizontal_wide_merge_tree.sql | 2 +- .../03037_dynamic_merges_2_vertical_compact_merge_tree.sql | 2 +- .../03037_dynamic_merges_2_vertical_wide_merge_tree.sql | 2 +- .../03038_nested_dynamic_merges_compact_horizontal.sql | 2 +- .../03038_nested_dynamic_merges_compact_vertical.sql | 2 +- .../03038_nested_dynamic_merges_wide_horizontal.sql | 2 +- .../03038_nested_dynamic_merges_wide_vertical.sql | 2 +- 12 files changed, 14 insertions(+), 13 deletions(-) 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 index b66fe5e2187..07371ee099b 100644 --- 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 @@ -2,7 +2,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); 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 index 8a376b6d7d7..2b55a31e937 100644 --- 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 @@ -2,7 +2,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); 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 index 127b56e727c..ea7295a9eab 100644 --- 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 @@ -2,7 +2,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -13,7 +13,8 @@ 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;; +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; 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 index e5c273cb592..e888a14b323 100644 --- 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 @@ -2,7 +2,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index 6d7a0dd8c18..e633b277ebd 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -3,7 +3,7 @@ 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; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index 011d54d2360..90dbc2d84f5 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -3,7 +3,7 @@ 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; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index 1a74f9e5417..ffd2618ee51 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -3,7 +3,7 @@ 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; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index cbc834e9660..36dff88751b 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -3,7 +3,7 @@ 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; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index ff1dc5e7ded..1d5c63dcdf1 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -6,7 +6,7 @@ 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;; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index f9b0101cb87..2bffe35c577 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -6,7 +6,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index 5f373d41c7d..fb686091ebb 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -6,7 +6,7 @@ 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; +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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index 36bbc76b8cb..ed195452d56 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -6,7 +6,7 @@ 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; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); From e0b125368855e57733132046de5cd383ccc9b7d2 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 25 Jul 2024 15:25:33 +0200 Subject: [PATCH 184/321] Fix harder --- src/AggregateFunctions/SingleValueData.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/SingleValueData.cpp b/src/AggregateFunctions/SingleValueData.cpp index 996e64b22e0..566b40253a3 100644 --- a/src/AggregateFunctions/SingleValueData.cpp +++ b/src/AggregateFunctions/SingleValueData.cpp @@ -195,7 +195,7 @@ bool SingleValueDataFixed::isEqualTo(const IColumn & column, size_t index) co template bool SingleValueDataFixed::isEqualTo(const SingleValueDataFixed & to) const { - return has() && to.value == value; + return has() && to.has() && to.value == value; } template @@ -905,7 +905,7 @@ template bool SingleValueDataNumeric::isEqualTo(const DB::SingleValueDataBase & to) const { auto const & other = assert_cast(to); - return memory.get().isEqualTo(other.memory.get()); + return to.has() && memory.get().isEqualTo(other.memory.get()); } template @@ -1291,7 +1291,7 @@ bool SingleValueDataGeneric::isEqualTo(const IColumn & column, size_t row_num) c bool SingleValueDataGeneric::isEqualTo(const DB::SingleValueDataBase & other) const { auto const & to = assert_cast(other); - return has() && to.value == value; + return has() && to.has() && to.value == value; } void SingleValueDataGeneric::set(const IColumn & column, size_t row_num, Arena *) From 64eeece5331c36c4f2b1c00c2bc229e40cd74d63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 15:49:59 +0200 Subject: [PATCH 185/321] fix --- tests/queries/0_stateless/02446_parent_zero_copy_locks.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql index 1cae8ae0237..a44322e02cf 100644 --- a/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql +++ b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql @@ -36,8 +36,8 @@ select 1, * from rmt1 order by n; system sync replica rmt1; select 2, * from rmt2 order by n; --- wait for outdated parts to be removed -select throwIf(count() = 0) from ( +-- wait for outdated parts to be removed (do not ignore _state column, so it will count Deleting parts as well) +select throwIf(count() = 0), groupArray(_state) from ( select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0 ) format Null; -- { retry 30 until serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } From f654db215ffb961010763c0daf6484fa75e4fd6b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 25 Jul 2024 15:56:29 +0200 Subject: [PATCH 186/321] Fix naming --- ....reference => 03210_fix_single_value_data_assertion.reference} | 0 ...ta-assertion.sql => 03210_fix_single_value_data_assertion.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03210_fix-single-value-data-assertion.reference => 03210_fix_single_value_data_assertion.reference} (100%) rename tests/queries/0_stateless/{03210_fix-single-value-data-assertion.sql => 03210_fix_single_value_data_assertion.sql} (100%) diff --git a/tests/queries/0_stateless/03210_fix-single-value-data-assertion.reference b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference similarity index 100% rename from tests/queries/0_stateless/03210_fix-single-value-data-assertion.reference rename to tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference diff --git a/tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql similarity index 100% rename from tests/queries/0_stateless/03210_fix-single-value-data-assertion.sql rename to tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql From 4b505badd3566cf2b47681c667ee134699cf2764 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Jul 2024 14:11:41 +0000 Subject: [PATCH 187/321] Attempt to fix flakiness of some window view tests --- .../queries/0_stateless/01052_window_view_proc_tumble_to_now.sh | 1 + tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh | 1 + tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh | 1 + tests/queries/0_stateless/01055_window_view_proc_hop_to.sh | 1 + .../0_stateless/01075_window_view_proc_tumble_to_now_populate.sh | 1 + 5 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index 4325ebeed24..5c70806ea7b 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh index 8e28995980f..32c9c52ab09 100755 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh index ee11b265ecd..ba566bb4ae6 100755 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh index ea8ad372617..0db4173b3dc 100755 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh index f7842af4dad..67c249a9d0e 100755 --- a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh +++ b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From baee31c12a68cd6e8f906a9224a39eea446a8f2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 16:15:37 +0200 Subject: [PATCH 188/321] fix truncate database --- src/Interpreters/InterpreterDropQuery.cpp | 3 +-- tests/queries/0_stateless/02842_truncate_database.reference | 2 ++ tests/queries/0_stateless/02842_truncate_database.sql | 4 ++++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b68b3ddcd48..bad3e5277db 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -399,10 +399,9 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (query.if_empty) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); - if (database->hasReplicationThread()) + if (!truncate && database->hasReplicationThread()) database->stopReplication(); - if (database->shouldBeEmptyOnDetach()) { /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. diff --git a/tests/queries/0_stateless/02842_truncate_database.reference b/tests/queries/0_stateless/02842_truncate_database.reference index 71f52bcd1da..bc8c0210d27 100644 --- a/tests/queries/0_stateless/02842_truncate_database.reference +++ b/tests/queries/0_stateless/02842_truncate_database.reference @@ -20,3 +20,5 @@ source_table_stripe_log source_table_tiny_log === DICTIONARIES IN test_truncate_database === dest_dictionary +new tables +new_table diff --git a/tests/queries/0_stateless/02842_truncate_database.sql b/tests/queries/0_stateless/02842_truncate_database.sql index 09ac844cfe2..be92108ccb8 100644 --- a/tests/queries/0_stateless/02842_truncate_database.sql +++ b/tests/queries/0_stateless/02842_truncate_database.sql @@ -73,4 +73,8 @@ SELECT * FROM dest_dictionary; -- {serverError UNKNOWN_TABLE} SHOW TABLES FROM test_truncate_database; SHOW DICTIONARIES FROM test_truncate_database; +CREATE TABLE new_table (x UInt16) ENGINE = ReplicatedMergeTree ORDER BY x; +select 'new tables'; +SHOW TABLES FROM test_truncate_database; + DROP DATABASE test_truncate_database; From 0642ed19b7c67e443be110f2a0f2d1f032ddd8d5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 14:17:45 +0000 Subject: [PATCH 189/321] Fixing more tests. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bdb90abd326..e5aeb9686be 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1525,7 +1525,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); + query_info.filter_actions_dag = std::make_shared(filter_actions_dag->clone()); buildIndexes( indexes, From 1973458ae07a5cd519b7069451d2be5822a89bf7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 15:00:37 +0000 Subject: [PATCH 190/321] Update PlannerWindowFunctions --- src/Planner/Planner.cpp | 13 ++++++------- src/Planner/PlannerWindowFunctions.cpp | 15 ++------------- src/Planner/PlannerWindowFunctions.h | 2 +- 3 files changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fb721069e6e..968642dc9de 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,19 +933,19 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, void addWindowSteps(QueryPlan & query_plan, const PlannerContextPtr & planner_context, - const WindowAnalysisResult & window_analysis_result) + WindowAnalysisResult & window_analysis_result) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - const auto & window_descriptions = window_analysis_result.window_descriptions; - auto perm = sortWindowDescriptions(window_descriptions); + auto & window_descriptions = window_analysis_result.window_descriptions; + sortWindowDescriptions(window_descriptions); size_t window_descriptions_size = window_descriptions.size(); for (size_t i = 0; i < window_descriptions_size; ++i) { - const auto & window_description = window_descriptions[perm[i]]; + const auto & window_description = window_descriptions[i]; /** We don't need to sort again if the input from previous window already * has suitable sorting. Also don't create sort steps when there are no @@ -958,9 +958,8 @@ void addWindowSteps(QueryPlan & query_plan, bool need_sort = !window_description.full_sort_description.empty(); if (need_sort && i != 0) { - auto prev = perm[i - 1]; - need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[prev].full_sort_description) - || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[prev].partition_by.size()); + need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description) + || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[i - 1].partition_by.size()); } if (need_sort) { diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index ba0e11df76b..225852de5a7 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -122,7 +122,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & return result; } -std::vector sortWindowDescriptions(const std::vector & window_descriptions) +void sortWindowDescriptions(std::vector & window_descriptions) { auto window_description_comparator = [](const WindowDescription & lhs, const WindowDescription & rhs) { @@ -153,18 +153,7 @@ std::vector sortWindowDescriptions(const std::vector return left.size() > right.size(); }; - auto comparator = [&](size_t lhs, size_t rhs) - { - return window_description_comparator(window_descriptions[lhs], window_descriptions[rhs]); - }; - - std::vector perm(window_descriptions.size()); - for (size_t i = 0; i < perm.size(); ++i) - perm[i] = i; - - ::sort(perm.begin(), perm.end(), comparator); - - return perm; + ::sort(window_descriptions.begin(), window_descriptions.end(), window_description_comparator); } } diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h index 3039ecefc4b..1552ef5a71f 100644 --- a/src/Planner/PlannerWindowFunctions.h +++ b/src/Planner/PlannerWindowFunctions.h @@ -15,6 +15,6 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & /** Try to sort window descriptions in such an order that the window with the longest * sort description goes first, and all window that use its prefixes follow. */ -std::vector sortWindowDescriptions(const std::vector & window_descriptions); +void sortWindowDescriptions(std::vector & window_descriptions); } From 93a2dbf85c446cd5efa802dfe8fc99e2a82f394f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 Jul 2024 17:02:11 +0200 Subject: [PATCH 191/321] 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 fb271436a1efe969f4de09b14aec942baa145cb9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 15:37:13 +0000 Subject: [PATCH 192/321] Remove ActionsDAGPtr completely. --- src/Interpreters/ActionsDAG.h | 3 --- src/Interpreters/evaluateConstantExpression.cpp | 4 ++-- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +++--- src/Processors/QueryPlan/SourceStepWithFilter.h | 6 +++--- src/Processors/QueryPlan/TotalsHavingStep.h | 3 --- src/Processors/QueryPlan/WindowStep.h | 3 --- src/Storages/MergeTree/KeyCondition.cpp | 14 +++++++------- src/Storages/MergeTree/KeyCondition.h | 2 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 3 +++ src/Storages/SelectQueryInfo.h | 3 --- src/Storages/StorageMerge.cpp | 2 +- 11 files changed, 20 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 76cc9327530..43c1b41a240 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -11,9 +11,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class IExecutableFunction; using ExecutableFunctionPtr = std::shared_ptr; diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 42d6f3d3037..4bfc80af1fe 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -679,9 +679,9 @@ std::optional evaluateExpressionOverConstantCondition( size_t max_elements) { auto inverted_dag = KeyCondition::cloneASTWithInversionPushDown({predicate}, context); - auto matches = matchTrees(expr, *inverted_dag, false); + auto matches = matchTrees(expr, inverted_dag, false); - auto predicates = analyze(inverted_dag->getOutputs().at(0), matches, context, max_elements); + auto predicates = analyze(inverted_dag.getOutputs().at(0), matches, context, max_elements); if (!predicates) return {}; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 63b4e019066..f53212407d2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->prewhere_actions->clone()), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->row_level_filter->clone()), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(std::make_unique(filter_step->getExpression().clone()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(filter_step->getExpression().clone(), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index f7a030c0628..6cea5fd7245 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -45,9 +45,9 @@ public: const Names & requiredSourceColumns() const { return required_source_columns; } - void addFilter(ActionsDAGPtr filter_dag, std::string column_name) + void addFilter(ActionsDAG filter_dag, std::string column_name) { - filter_nodes.nodes.push_back(&filter_dag->findInOutputs(column_name)); + filter_nodes.nodes.push_back(&filter_dag.findInOutputs(column_name)); filter_dags.push_back(std::move(filter_dag)); } @@ -86,7 +86,7 @@ protected: private: /// Will be cleared after applyFilters() is called. ActionDAGNodes filter_nodes; - std::vector filter_dags; + std::vector filter_dags; }; } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 927b8d99de3..4b414d41c57 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -6,9 +6,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - enum class TotalsMode : uint8_t; /// Execute HAVING and calculate totals. See TotalsHavingTransform. diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index 47883e5edf6..d79cd7fd45e 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -6,9 +6,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class WindowTransform; class WindowStep : public ITransformingStep diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 0eb59a47cae..69bffac9160 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -696,22 +696,22 @@ const std::unordered_map KeyConditi {"hilbertEncode", SpaceFillingCurveType::Hilbert} }; -ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) +ActionsDAG KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { - auto res = std::make_unique(); + ActionsDAG res; std::unordered_map to_inverted; for (auto & node : nodes) - node = &DB::cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false); + node = &DB::cloneASTWithInversionPushDown(*node, res, to_inverted, context, false); if (nodes.size() > 1) { auto function_builder = FunctionFactory::instance().get("and", context); - nodes = {&res->addFunction(function_builder, std::move(nodes), "")}; + nodes = {&res.addFunction(function_builder, std::move(nodes), "")}; } - res->getOutputs().swap(nodes); + res.getOutputs().swap(nodes); return res; } @@ -826,9 +826,9 @@ KeyCondition::KeyCondition( * are pushed down and applied (when possible) to leaf nodes. */ auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context); - assert(inverted_dag->getOutputs().size() == 1); + assert(inverted_dag.getOutputs().size() == 1); - const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0]; + const auto * inverted_dag_filter_node = inverted_dag.getOutputs()[0]; RPNBuilder builder(inverted_dag_filter_node, context, [&](const RPNBuilderTreeNode & node, RPNElement & out) { diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index a9e1a589ba5..e9343ec08ea 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -134,7 +134,7 @@ public: DataTypePtr current_type, bool single_point = false); - static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context); + static ActionsDAG cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context); bool matchesExactContinuousRange() const; diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 116edf5b9cb..1d0569e0df6 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -10,6 +10,9 @@ namespace DB { +class ActionsDAG; +using ActionsDAGPtr = std::unique_ptr; + namespace ErrorCodes { extern const int LOGICAL_ERROR; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 60f103fdb70..1c4cb7d92d8 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -18,9 +18,6 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - struct PrewhereInfo; using PrewhereInfoPtr = std::shared_ptr; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0e1568c8e79..e5de15c1d21 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1279,7 +1279,7 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(std::make_unique(actions_dag.clone()), filter_column_name); + step->addFilter(actions_dag.clone(), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const From ccd92d20821903123d4748027cc2248095b34efa Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Jul 2024 17:44:26 +0200 Subject: [PATCH 193/321] Update chassert in cache --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 198f6c0ea04..c928d25c7b8 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -810,6 +810,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { last_caller_id = FileSegment::getCallerId(); + chassert(file_offset_of_buffer_end <= read_until_position); if (file_offset_of_buffer_end == read_until_position) return false; @@ -1051,7 +1052,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (download_current_segment && download_current_segment_succeeded) chassert(file_segment.getCurrentWriteOffset() >= file_offset_of_buffer_end); - chassert(file_offset_of_buffer_end <= read_until_position); + + chassert( + file_offset_of_buffer_end <= read_until_position, + fmt::format("Expected {} <= {} (size: {}, read range: {})", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); } swap(*implementation_buffer); From e199fbaeaadd05b28e9dee1265fc813b081071f1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 25 Jul 2024 17:55:32 +0200 Subject: [PATCH 194/321] Update tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 620281ee972..13aa64d3cbe 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -168,7 +168,7 @@ fi START_TIME=$(get_now) STOP_TIME=$((START_TIME + MAIN_TIME_PART)) SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) -MIN_ITERATIONS=25 +MIN_ITERATIONS=20 run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 1 & PID_1=$! run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 2 & PID_2=$! From 869f6a6f105f50aa4d0e71e6440646b78539f0ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 16:33:12 +0000 Subject: [PATCH 195/321] Updating PrewhereInfo --- src/Interpreters/ExpressionAnalyzer.cpp | 9 ++++----- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++-------- src/Planner/PlannerJoinTree.cpp | 8 +++----- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 8 ++++---- .../optimizePrimaryKeyConditionAndLimit.cpp | 2 +- .../Optimizations/optimizeReadInOrder.cpp | 12 +++++------- .../Optimizations/projectionsCommon.cpp | 15 ++++++--------- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++-------- .../QueryPlan/SourceStepWithFilter.cpp | 9 +++------ src/Storages/IStorage.cpp | 3 +-- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 ++-- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 6 +++--- src/Storages/SelectQueryInfo.h | 7 +++---- src/Storages/StorageBuffer.cpp | 7 +++---- 16 files changed, 52 insertions(+), 70 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 5972d89bddd..d25434a515d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -2230,12 +2230,11 @@ void ExpressionAnalysisResult::checkActions() const /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (hasPrewhere()) { - auto check_actions = [](const std::optional & actions) + auto check_actions = [](ActionsDAG & actions) { - if (actions) - for (const auto & node : actions->getNodes()) - if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) - throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action"); + for (const auto & node : actions.getNodes()) + if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action"); }; check_actions(prewhere_info->prewhere_actions); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 174e6b5b0e0..4fd6f7a2900 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -937,7 +937,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() { { const auto & node - = query_info_copy.prewhere_info->prewhere_actions->findInOutputs(query_info_copy.prewhere_info->prewhere_column_name); + = query_info_copy.prewhere_info->prewhere_actions.findInOutputs(query_info_copy.prewhere_info->prewhere_column_name); added_filter_nodes.nodes.push_back(&node); } @@ -1058,7 +1058,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header); + header = analysis_result.prewhere_info->prewhere_actions.updateHeader(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); } @@ -1521,7 +1521,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->prewhere_actions->clone(), + expressions.prewhere_info->prewhere_actions.clone(), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -2066,7 +2066,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c }); } - auto filter_actions = std::make_shared(prewhere_info.prewhere_actions->clone()); + auto filter_actions = std::make_shared(prewhere_info.prewhere_actions.clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2157,7 +2157,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() if (prewhere_info) { /// Get some columns directly from PREWHERE expression actions - auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); + auto prewhere_required_columns = prewhere_info->prewhere_actions.getRequiredColumns().getNames(); columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); if (prewhere_info->row_level_filter) @@ -2229,7 +2229,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() if (prewhere_info) { NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end()); - Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns(); + Block prewhere_actions_result = prewhere_info->prewhere_actions.getResultColumns(); /// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards. /// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure. @@ -2268,7 +2268,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() { /// Don't remove columns which are needed to be aliased. for (const auto & name : required_columns) - prewhere_info->prewhere_actions->tryRestoreColumn(name); + prewhere_info->prewhere_actions.tryRestoreColumn(name); /// Add physical columns required by prewhere actions. for (const auto & column : required_columns_from_prewhere) @@ -2326,7 +2326,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle if (analysis_result.hasPrewhere()) { auto & prewhere_info = analysis_result.prewhere_info; - filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name)); + filter_nodes.push_back(&prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name)); if (prewhere_info->row_level_filter) filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name)); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index e9f886ab162..a3db0395ccc 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -437,7 +437,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info std::unordered_set required_output_nodes; - for (const auto * input : prewhere_actions->getInputs()) + for (const auto * input : prewhere_actions.getInputs()) { if (required_columns.contains(input->result_name)) required_output_nodes.insert(input); @@ -446,7 +446,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info if (required_output_nodes.empty()) return; - auto & prewhere_outputs = prewhere_actions->getOutputs(); + auto & prewhere_outputs = prewhere_actions.getOutputs(); for (const auto & output : prewhere_outputs) { auto required_output_node_it = required_output_nodes.find(output); @@ -801,10 +801,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name))) { if (!prewhere_info) - prewhere_info = std::make_shared(); - - if (!prewhere_info->prewhere_actions) { + prewhere_info = std::make_shared(); prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 5711189136c..dc73521210a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -56,7 +56,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) return; const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); - if (storage_prewhere_info && storage_prewhere_info->prewhere_actions) + if (storage_prewhere_info) return; /// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans. @@ -165,16 +165,16 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) { prewhere_info->prewhere_column_name = conditions.front()->result_name; if (prewhere_info->remove_prewhere_column) - prewhere_info->prewhere_actions->getOutputs().push_back(conditions.front()); + prewhere_info->prewhere_actions.getOutputs().push_back(conditions.front()); } else { prewhere_info->remove_prewhere_column = true; FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {}); + const auto * node = &prewhere_info->prewhere_actions.addFunction(func_builder_and, std::move(conditions), {}); prewhere_info->prewhere_column_name = node->result_name; - prewhere_info->prewhere_actions->getOutputs().push_back(node); + prewhere_info->prewhere_actions.getOutputs().push_back(node); } source_step_with_filter->updatePrewhereInfo(prewhere_info); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index f53212407d2..490b79fbf8d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,7 +18,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions.clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 252420e19fe..99df6da263f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -191,13 +191,11 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi /// Should ignore limit if there is filtering. limit = 0; - if (prewhere_info->prewhere_actions) - { - //std::cerr << "====== Adding prewhere " << std::endl; - appendExpression(dag, *prewhere_info->prewhere_actions); - if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) - appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); - } + //std::cerr << "====== Adding prewhere " << std::endl; + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); + } return; } diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 571d1dd0cc1..7414d479cc9 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -128,15 +128,12 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & return false; } - if (prewhere_info->prewhere_actions) - { - appendExpression(*prewhere_info->prewhere_actions); - if (const auto * filter_expression - = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) - filter_nodes.push_back(filter_expression); - else - return false; - } + appendExpression(prewhere_info->prewhere_actions); + if (const auto * filter_expression + = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + filter_nodes.push_back(filter_expression); + else + return false; } return true; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e5aeb9686be..483876dd293 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -109,8 +109,7 @@ bool restorePrewhereInputs(PrewhereInfo & info, const NameSet & inputs) if (info.row_level_filter) added = added || restoreDAGInputs(*info.row_level_filter, inputs); - if (info.prewhere_actions) - added = added || restoreDAGInputs(*info.prewhere_actions, inputs); + added = added || restoreDAGInputs(info.prewhere_actions, inputs); return added; } @@ -175,9 +174,8 @@ static void updateSortDescriptionForOutputStream( Block original_header = output_stream.header.cloneEmpty(); if (prewhere_info) { - if (prewhere_info->prewhere_actions) { - FindOriginalNodeForOutputName original_column_finder(*prewhere_info->prewhere_actions); + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -2131,7 +2129,6 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const prefix.push_back(format_settings.indent_char); prefix.push_back(format_settings.indent_char); - if (prewhere_info->prewhere_actions) { format_settings.out << prefix << "Prewhere filter" << '\n'; format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name; @@ -2139,7 +2136,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); expression->describeActions(format_settings.out, prefix); } @@ -2169,12 +2166,11 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_info_map = std::make_unique(); prewhere_info_map->add("Need filter", prewhere_info->need_filter); - if (prewhere_info->prewhere_actions) { std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index b91debc8239..3de9ae37db0 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -34,9 +34,8 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo block.erase(prewhere_info->row_level_column_name); } - if (prewhere_info->prewhere_actions) { - block = prewhere_info->prewhere_actions->updateHeader(block); + block = prewhere_info->prewhere_actions.updateHeader(block); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -102,7 +101,6 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con prefix.push_back(format_settings.indent_char); prefix.push_back(format_settings.indent_char); - if (prewhere_info->prewhere_actions) { format_settings.out << prefix << "Prewhere filter" << '\n'; format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name; @@ -110,7 +108,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); expression->describeActions(format_settings.out, prefix); } @@ -132,12 +130,11 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_info_map = std::make_unique(); prewhere_info_map->add("Need filter", prewhere_info->need_filter); - if (prewhere_info->prewhere_actions) { std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 823a6ae1cbc..755d71df531 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -325,9 +325,8 @@ std::string PrewhereInfo::dump() const ss << "row_level_filter " << row_level_filter->dumpDAG() << "\n"; } - if (prewhere_actions) { - ss << "prewhere_actions " << prewhere_actions->dumpDAG() << "\n"; + ss << "prewhere_actions " << prewhere_actions.dumpDAG() << "\n"; } ss << "remove_prewhere_column " << remove_prewhere_column diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 26595fbb36d..a9b77fb6c03 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -329,7 +329,7 @@ void MergeTreePrefetchedReadPool::fillPerPartStatistics() part_stat.sum_marks += range.end - range.begin; const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info - ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + ? prewhere_info->prewhere_actions.getRequiredColumnsNames() : column_names; part_stat.approx_size_of_mark = getApproximateSizeOfGranule(*read_info.data_part, columns); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 46482bc0959..6d2560bc9c7 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -65,7 +65,7 @@ static size_t calculateMinMarksPerTask( /// 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() + ? prewhere_info->prewhere_actions.getRequiredColumnsNames() : columns_to_read; const size_t part_compressed_bytes = getApproxSizeOfPart(*part.data_part, columns); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index f1df9e231c4..1a0709faf1c 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -59,7 +59,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( if (prewhere_info) LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}", - (prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("")), + prewhere_info->prewhere_actions.dumpDAG(), (!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string(""))); } @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions.clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 1d0569e0df6..36ff6c0a4bd 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -216,11 +216,11 @@ const ActionsDAG::Node & addAndTrue( /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere) { - if (!prewhere_info || !prewhere_info->prewhere_actions) + if (!prewhere_info) return true; /// 1. List all condition nodes that are combined with AND into PREWHERE condition - const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + const auto & condition_root = prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name); const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); if (!is_conjunction) return false; @@ -306,7 +306,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } /// 6. Find all outputs of the original DAG - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + auto original_outputs = prewhere_info->prewhere_actions.getOutputs(); /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 NameSet all_output_names; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 1c4cb7d92d8..7ad6a733c6f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -45,7 +45,7 @@ struct PrewhereInfo /// This actions are separate because prewhere condition should not be executed over filtered rows. std::optional row_level_filter; /// Actions which are executed on block in order to get filter column for prewhere step. - std::optional prewhere_actions; + ActionsDAG prewhere_actions; String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; @@ -53,7 +53,7 @@ struct PrewhereInfo bool generated_by_optimizer = false; PrewhereInfo() = default; - explicit PrewhereInfo(std::optional prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(ActionsDAG prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} std::string dump() const; @@ -65,8 +65,7 @@ struct PrewhereInfo if (row_level_filter) prewhere_info->row_level_filter = row_level_filter->clone(); - if (prewhere_actions) - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = prewhere_actions.clone(); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index aee4e4683ad..04e6d6676d1 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -319,13 +319,12 @@ void StorageBuffer::read( src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); } - if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( actions_dag.clone(), - std::move(*src_table_query_info.prewhere_info->prewhere_actions)); + std::move(src_table_query_info.prewhere_info->prewhere_actions)); - src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); + src_table_query_info.prewhere_info->prewhere_actions.removeUnusedActions(); } } @@ -440,7 +439,7 @@ void StorageBuffer::read( }); } - auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings); + auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions.clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( From 638d4640959f93924cec00b172d1cc1837d9ac10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 18:42:16 +0200 Subject: [PATCH 196/321] Fix test `00673_subquery_prepared_set_performance` --- .../0_stateless/00673_subquery_prepared_set_performance.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql b/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql index 0591592344c..b938d54c646 100644 --- a/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql +++ b/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql @@ -1,3 +1,5 @@ +-- Tags: no-tsan + DROP TABLE IF EXISTS mergetree_00673; CREATE TABLE mergetree_00673 (x UInt64) ENGINE = MergeTree ORDER BY x; From 7a003237befaa8d58cb6a77bb47e11fd1493e277 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 18:43:11 +0200 Subject: [PATCH 197/321] Fix test `00673_subquery_prepared_set_performance` --- .../0_stateless/00673_subquery_prepared_set_performance.sql | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql b/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql index b938d54c646..98c0802ffbc 100644 --- a/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql +++ b/tests/queries/0_stateless/00673_subquery_prepared_set_performance.sql @@ -1,14 +1,12 @@ --- Tags: no-tsan - DROP TABLE IF EXISTS mergetree_00673; CREATE TABLE mergetree_00673 (x UInt64) ENGINE = MergeTree ORDER BY x; INSERT INTO mergetree_00673 VALUES (1); -SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(10000000)))))))))))); +SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(1000000)))))))))))))))))))))); SET force_primary_key = 1; -SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(10000000)))))))))))); +SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(1000000)))))))))))))))))))))); DROP TABLE mergetree_00673; From 21f3a08ba7d626b967d99f694b1fde93da022ab9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 18:54:51 +0200 Subject: [PATCH 198/321] fix flaky test --- .../queries/0_stateless/03145_non_loaded_projection_backup.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 7df2118ad0c..95aef9bbc5b 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -6,8 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -nm -q " drop table if exists tp_1; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100); -system stop merges tp_1; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100) settings max_parts_to_merge_at_once=1; insert into tp_1 select number, number from numbers(3); set mutations_sync = 2; @@ -39,7 +38,6 @@ $CLICKHOUSE_CLIENT -nm -q " set send_logs_level='fatal'; drop table tp_1; restore table tp_1 from Disk('backups', '$backup_id'); -system stop merges tp_1; " | grep -o "RESTORED" $CLICKHOUSE_CLIENT -q "select count() from tp_1;" From f4b943f9f82bd4d297574774173e45abb2ee42d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 19:05:41 +0200 Subject: [PATCH 199/321] Fix tidy --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 377f6b36888..ba864035777 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -261,7 +261,7 @@ std::optional ReadBufferFromAzureBlobStorage::tryGetFileSize() if (!file_size) file_size = blob_client->GetProperties().Value.BlobSize; - return *file_size; + return file_size; } size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t range_begin, const std::function & /*progress_callback*/) const From a06df0729ea398642b715bfd2b121b1db0c5dd6d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 17:10:59 +0000 Subject: [PATCH 200/321] Remove the comment. --- src/Interpreters/ActionsDAG.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 85b2b38da17..4aaecc491e0 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3111,7 +3111,6 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs( } FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_) - //: actions(actions_) { const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) From 257be35365b8e0fd6163af027bbc02288ce8910b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 19:21:31 +0200 Subject: [PATCH 201/321] Minor tweaks and extra type tests --- src/AggregateFunctions/SingleValueData.cpp | 6 +++++- ..._fix_single_value_data_assertion.reference | 12 +++++++++++ .../03210_fix_single_value_data_assertion.sql | 20 ++++++++++++++++++- 3 files changed, 36 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/SingleValueData.cpp b/src/AggregateFunctions/SingleValueData.cpp index 566b40253a3..11931acbbc8 100644 --- a/src/AggregateFunctions/SingleValueData.cpp +++ b/src/AggregateFunctions/SingleValueData.cpp @@ -904,8 +904,9 @@ bool SingleValueDataNumeric::isEqualTo(const DB::IColumn & column, size_t ind template bool SingleValueDataNumeric::isEqualTo(const DB::SingleValueDataBase & to) const { + /// to.has() is checked in memory.get().isEqualTo auto const & other = assert_cast(to); - return to.has() && memory.get().isEqualTo(other.memory.get()); + return memory.get().isEqualTo(other.memory.get()); } template @@ -917,6 +918,7 @@ void SingleValueDataNumeric::set(const DB::IColumn & column, size_t row_num, template void SingleValueDataNumeric::set(const DB::SingleValueDataBase & to, DB::Arena * arena) { + /// to.has() is checked in memory.get().set auto const & other = assert_cast(to); return memory.get().set(other.memory.get(), arena); } @@ -924,6 +926,7 @@ void SingleValueDataNumeric::set(const DB::SingleValueDataBase & to, DB::Aren template bool SingleValueDataNumeric::setIfSmaller(const DB::SingleValueDataBase & to, DB::Arena * arena) { + /// to.has() is checked in memory.get().setIfSmaller auto const & other = assert_cast(to); return memory.get().setIfSmaller(other.memory.get(), arena); } @@ -931,6 +934,7 @@ bool SingleValueDataNumeric::setIfSmaller(const DB::SingleValueDataBase & to, template bool SingleValueDataNumeric::setIfGreater(const DB::SingleValueDataBase & to, DB::Arena * arena) { + /// to.has() is checked in memory.get().setIfGreater auto const & other = assert_cast(to); return memory.get().setIfGreater(other.memory.get(), arena); } diff --git a/tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference index e69de29bb2d..d8f7e13db55 100644 --- a/tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference +++ b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.reference @@ -0,0 +1,12 @@ +0 1 1 1 0 0 0 +1 3 3 3 2 2 2 +2 5 5 5 4 4 4 +3 7 7 7 6 6 6 +4 9 9 9 8 8 8 +5 11 11 11 10 10 10 +6 13 13 13 12 12 12 +7 15 15 15 14 14 14 +8 17 17 17 16 16 16 +9 19 19 19 18 18 18 + +0 107351244 107351244 107351244 107354520 107354520 107354520 diff --git a/tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql index 66e62377d6b..a1243ef0b25 100644 --- a/tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql +++ b/tests/queries/0_stateless/03210_fix_single_value_data_assertion.sql @@ -1 +1,19 @@ -SELECT intDiv(number, 2) AS k, count(toFixedString(toFixedString('hello', 5), 5)) IGNORE NULLS, sumArgMax(number, toString(number % 20)), argMax(toString(number), number) FROM (SELECT number FROM system.numbers LIMIT 65537) WHERE toLowCardinality(toLowCardinality(toNullable(21))) GROUP BY k WITH TOTALS ORDER BY k ASC NULLS FIRST LIMIT 255 SETTINGS group_by_overflow_mode = 'any', totals_mode = 'before_having', max_rows_to_group_by = 100000 FORMAT Null +SELECT + intDiv(number, 2) AS k, + sumArgMax(number, number % 20), + sumArgMax(number, leftPad(toString(number % 20), 5, '0')), -- Pad with 0 to preserve number ordering + sumArgMax(number, [number % 20, number % 20]), + sumArgMin(number, number % 20), + sumArgMin(number, leftPad(toString(number % 20), 5, '0')), + sumArgMin(number, [number % 20, number % 20]), +FROM +( + SELECT number + FROM system.numbers + LIMIT 65537 +) +GROUP BY k + WITH TOTALS +ORDER BY k ASC + LIMIT 10 +SETTINGS group_by_overflow_mode = 'any', totals_mode = 'before_having', max_rows_to_group_by = 100000; From ee193ffa019fc0f6104c1c5010ba00cd1993c8bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 19:25:58 +0200 Subject: [PATCH 202/321] 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 738d659e3bd8e222ff947e206d03d516c7053052 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 17:26:16 +0000 Subject: [PATCH 203/321] Do not remove constants from Distributed header if query is executed up to Complete. --- src/Storages/StorageDistributed.cpp | 7 ++- .../02563_analyzer_merge.reference | 1 + .../0_stateless/02563_analyzer_merge.sql | 45 +++++++++++++++++++ 3 files changed, 51 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 07892971ec2..9b417cda177 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -847,8 +847,11 @@ void StorageDistributed::read( /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ - for (auto & column : header) - column.column = column.column->convertToFullColumnIfConst(); + if (processed_stage != QueryProcessingStage::Complete) + { + for (auto & column : header) + column.column = column.column->convertToFullColumnIfConst(); + } modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); diff --git a/tests/queries/0_stateless/02563_analyzer_merge.reference b/tests/queries/0_stateless/02563_analyzer_merge.reference index 8be01c88d6f..2b3cc2d5dfb 100644 --- a/tests/queries/0_stateless/02563_analyzer_merge.reference +++ b/tests/queries/0_stateless/02563_analyzer_merge.reference @@ -1,2 +1,3 @@ 0 Value_0 02563_db test_merge_table_1 1 Value_1 02563_db test_merge_table_2 +91138316-5127-45ac-9c25-4ad8779777b4 160 diff --git a/tests/queries/0_stateless/02563_analyzer_merge.sql b/tests/queries/0_stateless/02563_analyzer_merge.sql index c90f7dcb2a5..217fb7019c4 100644 --- a/tests/queries/0_stateless/02563_analyzer_merge.sql +++ b/tests/queries/0_stateless/02563_analyzer_merge.sql @@ -35,4 +35,49 @@ SELECT id, value, _database, _table FROM 02563_db.test_merge_table ORDER BY id; DROP TABLE 02563_db.test_merge_table; DROP TABLE 02563_db.test_merge_table_1; DROP TABLE 02563_db.test_merge_table_2; + +CREATE TABLE 02563_db.t_1 +( + timestamp DateTime64(9), + a String, + b String +) +ENGINE = MergeTree +PARTITION BY formatDateTime(toStartOfMinute(timestamp), '%Y%m%d%H', 'UTC') +ORDER BY (timestamp, a, b); + +CREATE TABLE 02563_db.dist_t_1 (timestamp DateTime64(9), a String, b String) ENGINE = Distributed('test_shard_localhost', '02563_db', 't_1'); + +CREATE TABLE 02563_db.m ENGINE = Merge('02563_db', '^dist_'); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-13 22:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(30); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-13 23:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(30); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-14 00:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(100); + + +SELECT '91138316-5127-45ac-9c25-4ad8779777b4', + count() +FROM 02563_db.m; + +DROP TABLE 02563_db.t_1; +DROP TABLE 02563_db.dist_t_1; +DROP TABLE 02563_db.m; + DROP DATABASE 02563_db; From a3d5b2d29014bb3894982cdb1cadd65448ecdf63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 19:39:20 +0200 Subject: [PATCH 204/321] Update ZooKeeperImpl.cpp --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2728f953bea..d01fc341a63 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1014,9 +1014,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea LOG_INFO(log, "Finalizing session {}. finalization_started: {}, queue_finished: {}, reason: '{}'", session_id, already_started, requests_queue.isFinished(), reason); - /// Reset the original index. - original_index = -1; - auto expire_session_if_not_expired = [&] { /// No new requests will appear in queue after finish() From f32a0716b9bb42a09ece308a3ca64626099bfb1e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Jul 2024 19:45:06 +0200 Subject: [PATCH 205/321] Update 02842_truncate_database.sql --- tests/queries/0_stateless/02842_truncate_database.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02842_truncate_database.sql b/tests/queries/0_stateless/02842_truncate_database.sql index be92108ccb8..bcd818f55ba 100644 --- a/tests/queries/0_stateless/02842_truncate_database.sql +++ b/tests/queries/0_stateless/02842_truncate_database.sql @@ -73,7 +73,7 @@ SELECT * FROM dest_dictionary; -- {serverError UNKNOWN_TABLE} SHOW TABLES FROM test_truncate_database; SHOW DICTIONARIES FROM test_truncate_database; -CREATE TABLE new_table (x UInt16) ENGINE = ReplicatedMergeTree ORDER BY x; +CREATE TABLE new_table (x UInt16) ENGINE = MergeTree ORDER BY x; select 'new tables'; SHOW TABLES FROM test_truncate_database; From e46512a3bed4cd260042acbde4fcbef5cb83e032 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 25 Jul 2024 16:59:10 +0000 Subject: [PATCH 206/321] 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 eb4ec0912ad3a1e89ea7aec424366bc268262e11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 20:21:37 +0200 Subject: [PATCH 207/321] Rename bad setting --- CHANGELOG.md | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp | 4 ++-- tests/queries/0_stateless/03013_json_key_ignore_case.sh | 4 ++-- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a0933bd6544..07b37835dda 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -64,7 +64,7 @@ * 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)). +* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3f1ecc47f79..e10cf3fd745 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1122,7 +1122,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_ignore_key_case, false, "Ignore json key case while read json field from string", 0) \ + M(Bool, input_format_json_case_insensitive_column_matching, false, "Ignore case when matching JSON keys with CH columns", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index acd119c159b..9faf77e9087 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,7 +64,7 @@ static std::initializer_list Date: Thu, 25 Jul 2024 19:17:38 +0000 Subject: [PATCH 208/321] Fix: order by all with parallel replicas --- src/Analyzer/QueryTreeBuilder.cpp | 7 ++++++- ...09_parallel_replicas_order_by_all.reference | 12 ++++++++++++ .../03209_parallel_replicas_order_by_all.sql | 18 ++++++++++++++++++ 3 files changed, 36 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03209_parallel_replicas_order_by_all.reference create mode 100644 tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index a62b6e56ac5..ed1227b0f00 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -268,6 +268,8 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q } } + const auto enable_order_by_all = updated_context->getSettingsRef().enable_order_by_all; + auto current_query_tree = std::make_shared(std::move(updated_context), std::move(settings_changes)); current_query_tree->setIsSubquery(is_subquery); @@ -281,7 +283,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsGroupByWithRollup(select_query_typed.group_by_with_rollup); current_query_tree->setIsGroupByWithGroupingSets(select_query_typed.group_by_with_grouping_sets); current_query_tree->setIsGroupByAll(select_query_typed.group_by_all); - current_query_tree->setIsOrderByAll(select_query_typed.order_by_all); + /// order_by_all flag in AST is set w/o consideration of `enable_order_by_all` setting + /// since SETTINGS section has not been parsed yet, - so, check the setting here + if (enable_order_by_all) + current_query_tree->setIsOrderByAll(select_query_typed.order_by_all); current_query_tree->setOriginalAST(select_query); auto current_context = current_query_tree->getContext(); diff --git a/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.reference b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.reference new file mode 100644 index 00000000000..fd453d088a6 --- /dev/null +++ b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.reference @@ -0,0 +1,12 @@ +-- { echoOn } +SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=0; +B 3 10 +D 1 20 +A 2 30 +C \N 40 +SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=1; +B 3 10 +D 1 20 +A 2 30 +C \N 40 +DROP TABLE order_by_all SYNC; diff --git a/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql new file mode 100644 index 00000000000..46a3ab4d171 --- /dev/null +++ b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS order_by_all SYNC; +CREATE TABLE order_by_all +( + a String, + b Nullable(Int32), + all UInt64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_03210', 'r1') ORDER BY tuple(); + +INSERT INTO order_by_all VALUES ('B', 3, 10), ('C', NULL, 40), ('D', 1, 20), ('A', 2, 30); + +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='parallel_replicas'; +SET enable_order_by_all = 0; +-- { echoOn } +SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=0; +SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=1; + +DROP TABLE order_by_all SYNC; From 1ba4790511e1a06af8fb85e01767ce95866ee2a8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 25 Jul 2024 21:18:48 +0200 Subject: [PATCH 209/321] Review changes --- .../functions/type-conversion-functions.md | 550 ++++++++++-------- 1 file changed, 310 insertions(+), 240 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 057083d317f..844d957d538 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -51,7 +51,7 @@ SETTINGS cast_keep_nullable = 1 ## toInt8 -Converts an input value to a value of type `Int8`. +Converts an input value to a value of type [`Int8`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -61,10 +61,20 @@ toInt8(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: `SELECT toInt8(128) == -128;`, `SELECT toInt8(128.0) == -128;`, `SELECT toInt8('128') == -128;`. ::: **Returned value** @@ -72,11 +82,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 8-bit integer value. [Int8](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -106,32 +112,33 @@ Result: ## toInt8OrZero -Like [`toInt8`](#toint8), it takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `0`. +Like [`toInt8`](#toint8), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt8OrZero(expr) +toInt8OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise `0`. [Int8](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -160,32 +167,33 @@ Result: ## toInt8OrNull -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `NULL`. +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt8OrNull(expr) +toInt8OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise `NULL`. [Int8](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -212,7 +220,7 @@ Result: ## toInt8OrDefault -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns the default type value. +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -222,26 +230,28 @@ toInt8OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrDefault('0xc0fe', CAST('-1', 'Int8'));` +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise returns the default value. [Int8](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -268,7 +278,7 @@ Result: ## toInt16 -Converts an input value to a value of type `Int16`. +Converts an input value to a value of type [`Int16`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -278,10 +288,20 @@ toInt16(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: `SELECT toInt16(32768) == -32768;`, `SELECT toInt16(32768) == -32768;`, `SELECT toInt16('32768') == -32768;`. ::: **Returned value** @@ -289,11 +309,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 16-bit integer value. [Int16](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -323,32 +339,33 @@ Result: ## toInt16OrZero -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `0`. +Like [`toInt16`](#toint16), this function converts an input value to a value of type [Int16](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt16OrZero(expr) +toInt16OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise `0`. [Int16](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -377,32 +394,33 @@ Result: ## toInt16OrNull -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `NULL`. +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt16OrNull(expr) +toInt16OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise `NULL`. [Int16](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -431,7 +449,7 @@ Result: ## toInt16OrDefault -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns the default type value. +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -441,26 +459,28 @@ toInt16OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrDefault('0xc0fe', CAST('-1', 'Int16'));` +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise returns the default value. [Int16](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -485,7 +505,7 @@ Result: ## toInt32 -Converts an input value to a value of type `Int32`. +Converts an input value to a value of type [`Int32`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -495,10 +515,25 @@ toInt32(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: +``` +SELECT toInt32(2147483648) == -2147483648; +SELECT toInt32(2147483648.0) == -2147483648; +SELECT toInt32('2147483648') == -2147483648; +``` ::: **Returned value** @@ -506,11 +541,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 32-bit integer value. [Int32](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -540,32 +571,34 @@ Result: ## toInt32OrZero -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `0`. +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt32OrZero(expr) +toInt32OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: **Returned value** - 32-bit integer value if successful, otherwise `0`. [Int32](../data-types/int-uint.md) :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. ::: **Example** @@ -588,35 +621,36 @@ Result: - [`toInt32`](#toint32). - [`toInt32OrNull`](#toint32ornull). - [`toInt32OrDefault`](#toint32ordefault). -- + ## toInt32OrNull -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `NULL`. +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt32OrNull(expr) +toInt32OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 32-bit integer value if successful, otherwise `NULL`. [Int32](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -643,7 +677,7 @@ Result: ## toInt32OrDefault -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns the default type value. +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -653,24 +687,26 @@ toInt32OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrDefault('0xc0fe', CAST('-1', 'Int32'));` +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 32-bit integer value if successful, otherwise returns the default value. [Int32](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. - ::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. ::: **Example** @@ -697,7 +733,7 @@ Result: ## toInt64 -Converts an input value to a value of type `Int64`. +Converts an input value to a value of type [`Int64`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -707,10 +743,26 @@ toInt64(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: + +``` +SELECT toInt64(9223372036854775808) == -9223372036854775808; +SELECT toInt64(9223372036854775808.0) == -9223372036854775808; +SELECT toInt64('9223372036854775808') == --9223372036854775808; +``` ::: **Returned value** @@ -718,11 +770,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -752,32 +800,33 @@ Result: ## toInt64OrZero -Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `0`. +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt64OrZero(expr) +toInt64OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 64-bit integer value if successful, otherwise `0`. [Int64](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -806,32 +855,33 @@ Result: ## toInt64OrNull -Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `NULL`. +Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt64OrNull(expr) +toInt64OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -860,7 +910,7 @@ Result: ## toInt64OrDefault -Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns the default type value. +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns the default type value. **Syntax** @@ -870,24 +920,26 @@ toInt64OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrDefault('0xc0fe', CAST('-1', 'Int64'));` +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. - ::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. ::: **Example** @@ -916,7 +968,7 @@ Result: ## toInt128 -Converts an input value to a value of type `Int128`. +Converts an input value to a value of type [`Int128`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -926,10 +978,19 @@ toInt128(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), the result over or under flows. This is not considered an error. ::: **Returned value** @@ -937,11 +998,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 128-bit integer value. [Int128](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -971,7 +1028,7 @@ Result: ## toInt128OrZero -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `0`. +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -981,22 +1038,23 @@ toInt128OrZero(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise `0`. [Int128](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1025,32 +1083,33 @@ Result: ## toInt128OrNull -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `NULL`. +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt128OrNull(expr) +toInt128OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise `NULL`. [Int128](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1079,7 +1138,7 @@ Result: ## toInt128OrDefault -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns the default type value. +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -1089,26 +1148,28 @@ toInt128OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int128'));` +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise returns the default value. [Int128](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -1135,7 +1196,7 @@ Result: ## toInt256 -Converts an input value to a value of type `Int256`. +Converts an input value to a value of type [`Int256`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -1145,10 +1206,19 @@ toInt256(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), the result over or under flows. This is not considered an error. ::: **Returned value** @@ -1156,11 +1226,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 256-bit integer value. [Int256](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1190,32 +1256,33 @@ Result: ## toInt256OrZero -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `0`. +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt256OrZero(expr) +toInt256OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise `0`. [Int256](../data-types/int-uint.md). :::note -Functions uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1244,32 +1311,33 @@ Result: ## toInt256OrNull -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `NULL`. +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt256OrNull(expr) +toInt256OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise `NULL`. [Int256](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1298,7 +1366,7 @@ Result: ## toInt256OrDefault -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns the default type value. +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -1308,26 +1376,28 @@ toInt256OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int256'));` +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise returns the default value. [Int256](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: From 3c1004aee4a3b1f3e1b0bd91a1b02c6c9e16c832 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 25 Jul 2024 21:26:14 +0200 Subject: [PATCH 210/321] Fix typo --- .../functions/type-conversion-functions.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 844d957d538..4326753216e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -131,7 +131,7 @@ Types for which `0` is returned: - String representations of ordinary Float32/64 values. - String representations of Float values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), and the result over or under flows. +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. **Returned value** @@ -186,7 +186,7 @@ Types for which `\N` is returned: - String representations of ordinary Float32/64 values. - String representations of Float values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), and the result over or under flows. +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. **Returned value** @@ -300,7 +300,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: `SELECT toInt16(32768) == -32768;`, `SELECT toInt16(32768) == -32768;`, `SELECT toInt16('32768') == -32768;`. ::: @@ -527,7 +527,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: ``` SELECT toInt32(2147483648) == -2147483648; @@ -755,7 +755,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: ``` @@ -1386,7 +1386,7 @@ Supported types: Types for which the default value is returned: - Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int256'));` +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrDefault('0xc0fe', CAST('-1', 'Int256'));` - If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** From 7b8c41818bcec1c567c85b15d916fd2e064b7482 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 25 Jul 2024 19:34:41 +0000 Subject: [PATCH 211/321] 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 212/321] 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. From 18fb7396f941fd5a7e3872788ab07a18731dc943 Mon Sep 17 00:00:00 2001 From: xc0derx <11428624+xc0derx@users.noreply.github.com> Date: Thu, 25 Jul 2024 23:21:30 +0200 Subject: [PATCH 213/321] fix broken links (compression codecs) --- docs/en/sql-reference/statements/alter/column.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index aa6f132e08e..2e9b0cf3080 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -36,7 +36,7 @@ These actions are described in detail below. ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after | FIRST] ``` -Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md/#codecs) and `default_expr` (see the section [Default expressions](/docs/en/sql-reference/statements/create/table.md/#create-default-values)). +Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md/#column_compression_codec) and `default_expr` (see the section [Default expressions](/docs/en/sql-reference/statements/create/table.md/#create-default-values)). If the `IF NOT EXISTS` clause is included, the query won’t return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. If you want to add a column to the beginning of the table use the `FIRST` clause. Otherwise, the column is added to the end of the table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions. @@ -155,7 +155,7 @@ This query changes the `name` column properties: - Column-level Settings -For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md/#codecs). +For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md/#column_compression_codec). For examples of columns TTL modifying, see [Column TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). From d0c4c4151c5e4bcb86b9417f4ab8cc71316404b5 Mon Sep 17 00:00:00 2001 From: Shri Bodas Date: Thu, 25 Jul 2024 14:24:28 -0700 Subject: [PATCH 214/321] Update keepermap.md Needs quotes around keeper path --- docs/en/engines/table-engines/special/keepermap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index 5559cc2c648..04a9a4b0d4e 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -54,7 +54,7 @@ CREATE TABLE keeper_map_table `v2` String, `v3` Float32 ) -ENGINE = KeeperMap(/keeper_map_table, 4) +ENGINE = KeeperMap('/keeper_map_table', 4) PRIMARY KEY key ``` From f3c88ff66707a50523ccef6e964f2fe78a711ace Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 03:56:02 +0200 Subject: [PATCH 215/321] Fix benign data race in ZooKeeper --- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 27 ++++++++--------- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 29 +++++++++++++++++-- src/Common/ZooKeeper/ZooKeeperImpl.h | 9 +++--- .../StorageSystemZooKeeperConnection.cpp | 10 +++++-- 7 files changed, 54 insertions(+), 27 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 2c6cbc4a5d5..ce7489a33e5 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -548,7 +548,7 @@ public: virtual bool isExpired() const = 0; /// Get the current connected node idx. - virtual Int8 getConnectedNodeIdx() const = 0; + virtual std::optional getConnectedNodeIdx() const = 0; /// Get the current connected host and port. virtual String getConnectedHostPort() const = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 2194ad015bf..562c313ac0e 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -39,7 +39,7 @@ public: ~TestKeeper() override; bool isExpired() const override { return expired; } - Int8 getConnectedNodeIdx() const override { return 0; } + std::optional getConnectedNodeIdx() const override { return 0; } String getConnectedHostPort() const override { return "TestKeeper:0000"; } int32_t getConnectionXid() const override { return 0; } int64_t getSessionID() const override { return 0; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 01bb508da95..1250e1273b9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -128,16 +128,15 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr ShuffleHosts shuffled_hosts = shuffleHosts(); impl = std::make_unique(shuffled_hosts, args, zk_log); - Int8 node_idx = impl->getConnectedNodeIdx(); + auto node_idx = impl->getConnectedNodeIdx(); if (args.chroot.empty()) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); - /// If the balancing strategy has an optimal node then it will be the first in the list - bool connected_to_suboptimal_node = node_idx != shuffled_hosts[0].original_index; + bool connected_to_suboptimal_node = node_idx && *node_idx != shuffled_hosts[0].original_index; bool respect_az = args.prefer_local_availability_zone && !args.client_availability_zone.empty(); bool may_benefit_from_reconnecting = respect_az || args.get_priority_load_balancing.hasOptimalNode(); if (connected_to_suboptimal_node && may_benefit_from_reconnecting) @@ -145,7 +144,7 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr auto reconnect_timeout_sec = getSecondsUntilReconnect(args); LOG_DEBUG(log, "Connected to a suboptimal ZooKeeper host ({}, index {})." " To preserve balance in ZooKeeper usage, this ZooKeeper session will expire in {} seconds", - impl->getConnectedHostPort(), node_idx, reconnect_timeout_sec); + impl->getConnectedHostPort(), *node_idx, reconnect_timeout_sec); auto reconnect_task_holder = DB::Context::getGlobalContextInstance()->getSchedulePool().createTask("ZKReconnect", [this, optimal_host = shuffled_hosts[0]]() { @@ -154,13 +153,15 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr LOG_DEBUG(log, "Trying to connect to a more optimal node {}", optimal_host.host); ShuffleHosts node{optimal_host}; std::unique_ptr new_impl = std::make_unique(node, args, zk_log); - Int8 new_node_idx = new_impl->getConnectedNodeIdx(); - /// Maybe the node was unavailable when getting AZs first time, update just in case - if (args.availability_zone_autodetect && availability_zones[new_node_idx].empty()) + if (auto new_node_idx = new_impl->getConnectedNodeIdx(); new_node_idx) { - availability_zones[new_node_idx] = new_impl->tryGetAvailabilityZone(); - LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[new_node_idx]); + /// Maybe the node was unavailable when getting AZs first time, update just in case + if (args.availability_zone_autodetect && availability_zones[*new_node_idx].empty()) + { + availability_zones[*new_node_idx] = new_impl->tryGetAvailabilityZone(); + LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[*new_node_idx]); + } } optimal_impl = std::move(new_impl); @@ -1525,7 +1526,7 @@ void ZooKeeper::setServerCompletelyStarted() zk->setServerCompletelyStarted(); } -Int8 ZooKeeper::getConnectedHostIdx() const +std::optional ZooKeeper::getConnectedHostIdx() const { return impl->getConnectedNodeIdx(); } @@ -1544,10 +1545,10 @@ String ZooKeeper::getConnectedHostAvailabilityZone() const { if (args.implementation != "zookeeper" || !impl) return ""; - Int8 idx = impl->getConnectedNodeIdx(); - if (idx < 0) + std::optional idx = impl->getConnectedNodeIdx(); + if (!idx) return ""; /// session expired - return availability_zones.at(idx); + return availability_zones.at(*idx); } size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 4ae2cfa6096..657c9cb2c03 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -620,7 +620,7 @@ public: void setServerCompletelyStarted(); - Int8 getConnectedHostIdx() const; + std::optional getConnectedHostIdx() const; String getConnectedHostPort() const; int32_t getConnectionXid() const; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2728f953bea..53c7a5728aa 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -536,7 +536,7 @@ void ZooKeeper::connect( compressed_out.emplace(*out, CompressionCodecFactory::instance().get("LZ4", {})); } - original_index = static_cast(node.original_index); + original_index.store(node.original_index); break; } catch (...) @@ -1014,8 +1014,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea LOG_INFO(log, "Finalizing session {}. finalization_started: {}, queue_finished: {}, reason: '{}'", session_id, already_started, requests_queue.isFinished(), reason); - /// Reset the original index. - original_index = -1; + original_index.store(-1); auto expire_session_if_not_expired = [&] { @@ -1534,6 +1533,30 @@ void ZooKeeper::close() } +std::optional ZooKeeper::getConnectedNodeIdx() const +{ + int8_t res = original_index.load(); + if (res == -1) + return std::nullopt; + else + return res; +} + +String ZooKeeper::getConnectedHostPort() const +{ + auto idx = getConnectedNodeIdx(); + if (idx) + return args.hosts[*idx]; + else + return ""; +} + +int32_t ZooKeeper::getConnectionXid() const +{ + return next_xid.load(); +} + + void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) { /// logOperationIfNeeded(...) uses zk_log and can be called from different threads, so we have to use atomic shared_ptr diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 0c88c35b381..39082cd14c1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -114,13 +114,12 @@ public: ~ZooKeeper() override; - /// If expired, you can only destroy the object. All other methods will throw exception. bool isExpired() const override { return requests_queue.isFinished(); } - Int8 getConnectedNodeIdx() const override { return original_index; } - String getConnectedHostPort() const override { return (original_index == -1) ? "" : args.hosts[original_index]; } - int32_t getConnectionXid() const override { return next_xid.load(); } + std::optional getConnectedNodeIdx() const override; + String getConnectedHostPort() const override; + int32_t getConnectionXid() const override; String tryGetAvailabilityZone() override; @@ -219,7 +218,7 @@ private: ACLs default_acls; zkutil::ZooKeeperArgs args; - Int8 original_index = -1; + std::atomic original_index{-1}; /// Fault injection void maybeInjectSendFault(); diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index ec29b84dac3..72a7ba38429 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -27,7 +28,7 @@ ColumnsDescription StorageSystemZooKeeperConnection::getColumnsDescription() /* 0 */ {"name", std::make_shared(), "ZooKeeper cluster's name."}, /* 1 */ {"host", std::make_shared(), "The hostname/IP of the ZooKeeper node that ClickHouse connected to."}, /* 2 */ {"port", std::make_shared(), "The port of the ZooKeeper node that ClickHouse connected to."}, - /* 3 */ {"index", std::make_shared(), "The index of the ZooKeeper node that ClickHouse connected to. The index is from ZooKeeper config."}, + /* 3 */ {"index", std::make_shared(std::make_shared()), "The index of the ZooKeeper node that ClickHouse connected to. The index is from ZooKeeper config. If not connected, this column is NULL."}, /* 4 */ {"connected_time", std::make_shared(), "When the connection was established."}, /* 5 */ {"session_uptime_elapsed_seconds", std::make_shared(), "Seconds elapsed since the connection was established."}, /* 6 */ {"is_expired", std::make_shared(), "Is the current connection expired."}, @@ -64,7 +65,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co /// For read-only snapshot type functionality, it's acceptable even though 'getZooKeeper' may cause data inconsistency. auto fill_data = [&](const String & name, const zkutil::ZooKeeperPtr zookeeper, MutableColumns & columns) { - Int8 index = zookeeper->getConnectedHostIdx(); + auto index = zookeeper->getConnectedHostIdx(); String host_port = zookeeper->getConnectedHostPort(); if (index != -1 && !host_port.empty()) { @@ -78,7 +79,10 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co columns[0]->insert(name); columns[1]->insert(host); columns[2]->insert(port); - columns[3]->insert(index); + if (index) + columns[3]->insert(*index); + else + columns[3]->insertDefault(); columns[4]->insert(connected_time); columns[5]->insert(uptime); columns[6]->insert(zookeeper->expired()); From d6fdf29679ece887567cba6fa43aee4c22c7d6f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 04:48:41 +0200 Subject: [PATCH 216/321] Remove too long unit test --- .../tests/gtest_archive_reader_and_writer.cpp | 42 ------------------- 1 file changed, 42 deletions(-) diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 898c7017e7d..06f8f53546b 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -492,48 +492,6 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk) } } -TEST_P(ArchiveReaderAndWriterTest, LargeFile) -{ - /// Make an archive. - std::string_view contents = "The contents of a.txt\n"; - int times = 10000000; - { - auto writer = createArchiveWriter(getPathToArchive()); - { - auto out = writer->writeFile("a.txt", times * contents.size()); - for (int i = 0; i < times; i++) - writeString(contents, *out); - out->finalize(); - } - writer->finalize(); - } - - /// Read the archive. - auto reader = createArchiveReader(getPathToArchive()); - - ASSERT_TRUE(reader->fileExists("a.txt")); - - auto file_info = reader->getFileInfo("a.txt"); - EXPECT_EQ(file_info.uncompressed_size, contents.size() * times); - EXPECT_GT(file_info.compressed_size, 0); - - { - auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true); - for (int i = 0; i < times; i++) - ASSERT_TRUE(checkString(String(contents), *in)); - } - - { - /// Use an enumerator. - auto enumerator = reader->firstFile(); - ASSERT_NE(enumerator, nullptr); - EXPECT_EQ(enumerator->getFileName(), "a.txt"); - EXPECT_EQ(enumerator->getFileInfo().uncompressed_size, contents.size() * times); - EXPECT_GT(enumerator->getFileInfo().compressed_size, 0); - EXPECT_FALSE(enumerator->nextFile()); - } -} - TEST(TarArchiveReaderTest, FileExists) { String archive_path = "archive.tar"; From 9c6026965d985ca0ffcf0ab789d09946bd37c569 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 04:55:53 +0200 Subject: [PATCH 217/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 17a5ed385d4..a62f22d4bd9 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -700,6 +700,14 @@ std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { return std::nullopt; } + catch (const NetException &) + { + return std::nullopt; + } + catch (const Poco::Net::NetException &) + { + return std::nullopt; + } } return file_info->last_modified; From 64ff5d7bc443cdb15fd0a5eec391d449a617b3f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 05:07:56 +0200 Subject: [PATCH 218/321] Fix `00705_drop_create_merge_tree` --- tests/queries/0_stateless/00705_drop_create_merge_tree.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index d7754091290..ea8b9d02e49 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error -nm 2>/dev/null & -yes 'DROP TABLE table;' | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error -nm 2>/dev/null & +yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & +yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & wait ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table" From 400f8e5b2116ab585312e578eee4d783b9d6783b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 09:33:46 +0200 Subject: [PATCH 219/321] Fix stacktrace cache --- src/Common/StackTrace.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 59a58ac027a..ff8765c9727 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -495,18 +495,19 @@ using StackTraceCacheBase = std::map Date: Fri, 26 Jul 2024 09:39:31 +0200 Subject: [PATCH 220/321] Fix --- .../01676_clickhouse_client_autocomplete.python | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python index 9072dfeb09f..0f35d259c7c 100644 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python @@ -59,13 +59,14 @@ def test_completion(program, argv, comp_word): output = output_b.decode() debug_log_fd.write(repr(output_b) + "\n") debug_log_fd.flush() - # fail fast if there is a bell character in the output, - # meaning no concise completion is found - if "\x07" in output: - print(f"{comp_word}: FAIL") - return while not comp_word in output: + # fail fast if there is a bell character in the output, + # meaning no concise completion is found + if "\x07" in output: + print(f"{comp_word}: FAIL") + return + output_b = os.read(master, 4096) output += output_b.decode() debug_log_fd.write(repr(output_b) + "\n") From 83dba7194f3467dc0f6e5499d65bda8a66fa8206 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 Jul 2024 09:55:09 +0200 Subject: [PATCH 221/321] Add deserialization of empty state --- .../03208_groupArrayIntersect_serialization.reference | 1 + .../0_stateless/03208_groupArrayIntersect_serialization.sql | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference index c3b6e0cd5b7..e84856c90fd 100644 --- a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference +++ b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.reference @@ -10,3 +10,4 @@ a [(['2','4','6','8','10'])] b [(['2','4','6','8','10'])] c [(['2','4','6','8','10'])] d [] +e [] diff --git a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql index e05f78a4051..1b3d48ce0c3 100644 --- a/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql +++ b/tests/queries/0_stateless/03208_groupArrayIntersect_serialization.sql @@ -39,3 +39,5 @@ INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', SELECT 'c', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10', '20']), tuple(['2', '4', '6', '8', '10', '14'])]); SELECT 'd', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; +INSERT INTO grouparray_string Select groupArrayIntersectState([]::Array(Tuple(Array(String)))); +SELECT 'e', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string; From ad4e807cf4bb3633616b01e3616844fe2108d59f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 08:12:01 +0000 Subject: [PATCH 222/321] Fix stupid crash. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4fd6f7a2900..41306a79198 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2055,9 +2055,9 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & prewhere_info = *query_info.prewhere_info; - auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); if (prewhere_info.row_level_filter) { + auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, From 498ae4358647dbff5fde2861a7113a9c9597930a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 26 Jul 2024 10:42:23 +0200 Subject: [PATCH 223/321] Make 02908_many_requests_to_system_replicas less stressful --- ...08_many_requests_to_system_replicas.reference | 14 +++++++------- .../02908_many_requests_to_system_replicas.sh | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference index f376bb87044..fdefd2e3466 100644 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference @@ -1,10 +1,10 @@ -Creating 300 tables -900 1 1 -900 1 1 -900 1 1 -900 1 1 -Making 200 requests to system.replicas +Creating 50 tables +150 1 1 +150 1 1 +150 1 1 +150 1 1 +Making 100 requests to system.replicas Query system.replicas while waiting for other concurrent requests to finish 0 -900 +150 1 diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index a247c99a818..81ba59fc591 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -7,8 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -NUM_TABLES=300 -CONCURRENCY=200 +NUM_TABLES=50 +CONCURRENCY=100 echo "Creating $NUM_TABLES tables" @@ -46,10 +46,10 @@ wait; # Check results with different max_block_size -$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase()' -$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=1' -$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=77' -$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=11111' +$CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase()' +$CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=1' +$CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=77' +$CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=11111' echo "Making $CONCURRENCY requests to system.replicas" @@ -70,8 +70,8 @@ wait; $CLICKHOUSE_CLIENT -nq " SYSTEM FLUSH LOGS; --- without optimisation there are ~350K zk requests -SELECT sum(ProfileEvents['ZooKeeperTransactions']) < 30000 +-- Check that number of ZK request is less then a half of (total replicas * concurrency) +SELECT sum(ProfileEvents['ZooKeeperTransactions']) < (${NUM_TABLES} * 3 * ${CONCURRENCY} / 2) FROM system.query_log WHERE current_database=currentDatabase() AND log_comment='02908_many_requests'; " From 0c5c23e78477636560cd09f17b91db79e420680f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 10:21:36 +0200 Subject: [PATCH 224/321] More fixes --- src/Common/Exception.cpp | 42 +++++++++++++++++++++++++++++---- src/Common/Exception.h | 31 ++++++++++++++++++------ src/Common/SignalHandlers.cpp | 2 +- src/Common/StackTrace.cpp | 2 +- src/Common/ThreadPool.cpp | 4 ++-- src/Loggers/OwnSplitChannel.cpp | 7 +++++- src/Loggers/OwnSplitChannel.h | 2 ++ 7 files changed, 73 insertions(+), 17 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 33befa64946..c4bd4fbd943 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -100,7 +101,7 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_) { if (terminate_on_any_exception) std::_Exit(terminate_status_code); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); } @@ -110,7 +111,7 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_) { if (terminate_on_any_exception) std::_Exit(terminate_status_code); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); handle_error_code(message(), code, remote, getStackFramePointers()); } @@ -119,7 +120,7 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) { if (terminate_on_any_exception) std::_Exit(terminate_status_code); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -133,7 +134,7 @@ Exception::Exception(CreateFromSTDTag, const std::exception & exc) { if (terminate_on_any_exception) std::_Exit(terminate_status_code); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -223,10 +224,38 @@ Exception::FramePointers Exception::getStackFramePointers() const } thread_local bool Exception::enable_job_stack_trace = false; -thread_local std::vector Exception::thread_frame_pointers = {}; +thread_local bool Exception::can_use_thread_frame_pointers = false; +thread_local Exception::ThreadFramePointers Exception::thread_frame_pointers; + +Exception::ThreadFramePointers::ThreadFramePointers() +{ + can_use_thread_frame_pointers = true; +} + +Exception::ThreadFramePointers::~ThreadFramePointers() +{ + can_use_thread_frame_pointers = false; +} + +Exception::ThreadFramePointersBase Exception::getThreadFramePointers() +{ + if (can_use_thread_frame_pointers) + return thread_frame_pointers.frame_pointers; + + return {}; +} + +void Exception::setThreadFramePointers(ThreadFramePointersBase frame_pointers) +{ + if (can_use_thread_frame_pointers) + thread_frame_pointers.frame_pointers = std::move(frame_pointers); +} static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message) { + if (!OwnSplitChannel::isLoggingEnabled()) + return; + try { PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); @@ -242,6 +271,9 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { + if (!OwnSplitChannel::isLoggingEnabled()) + return; + /// Under high memory pressure, new allocations throw a /// MEMORY_LIMIT_EXCEEDED exception. /// diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 4e54c411bf1..a4f55f41caa 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -10,7 +10,6 @@ #include #include -#include #include #include @@ -49,14 +48,14 @@ public: { if (terminate_on_any_exception) std::terminate(); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); } Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code) { if (terminate_on_any_exception) std::terminate(); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); message_format_string = msg.format_string; message_format_string_args = msg.format_string_args; } @@ -65,18 +64,36 @@ public: { if (terminate_on_any_exception) std::terminate(); - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); message_format_string = msg.format_string; message_format_string_args = msg.format_string_args; } /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution static thread_local bool enable_job_stack_trace; - static thread_local std::vector thread_frame_pointers; + static thread_local bool can_use_thread_frame_pointers; + /// Because of unknown order of static destructor calls, + /// thread_frame_pointers can already be uninitialized when a different destructor generates an exception. + /// To prevent such scenarios, a wrapper class is created and a function that will return empty vector + /// if its destructor is already called + using ThreadFramePointersBase = std::vector; + struct ThreadFramePointers + { + ThreadFramePointers(); + ~ThreadFramePointers(); + + ThreadFramePointersBase frame_pointers; + }; + + static ThreadFramePointersBase getThreadFramePointers(); + static void setThreadFramePointers(ThreadFramePointersBase frame_pointers); + /// Callback for any exception static std::function callback; protected: + static thread_local ThreadFramePointers thread_frame_pointers; + // used to remove the sensitive information from exceptions if query_masking_rules is configured struct MessageMasked { @@ -178,7 +195,7 @@ class ErrnoException : public Exception public: ErrnoException(std::string && msg, int code, int with_errno) : Exception(msg, code), saved_errno(with_errno) { - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); addMessage(", {}", errnoToString(saved_errno)); } @@ -187,7 +204,7 @@ public: requires std::is_convertible_v ErrnoException(int code, T && message) : Exception(message, code), saved_errno(errno) { - capture_thread_frame_pointers = thread_frame_pointers; + capture_thread_frame_pointers = getThreadFramePointers(); addMessage(", {}", errnoToString(saved_errno)); } diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 52c83d80121..e025e49e0a3 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -89,7 +89,7 @@ void signalHandler(int sig, siginfo_t * info, void * context) writePODBinary(*info, out); writePODBinary(signal_context, out); writePODBinary(stack_trace, out); - writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector{}, out); + writeVectorBinary(Exception::enable_job_stack_trace ? Exception::getThreadFramePointers() : std::vector{}, out); writeBinary(static_cast(getThreadId()), out); writePODBinary(current_thread, out); diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index ff8765c9727..76277cbc993 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -489,7 +489,7 @@ struct CacheEntry using CacheEntryPtr = std::shared_ptr; -static constinit std::atomic can_use_cache = false; +static constinit bool can_use_cache = false; using StackTraceCacheBase = std::map>; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 01f561d573f..c8f1ae99969 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -51,7 +51,7 @@ public: if (!capture_frame_pointers) return; /// Save all previous jobs call stacks and append with current - frame_pointers = DB::Exception::thread_frame_pointers; + frame_pointers = DB::Exception::getThreadFramePointers(); frame_pointers.push_back(StackTrace().getFramePointers()); } @@ -455,7 +455,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ try { if (DB::Exception::enable_job_stack_trace) - DB::Exception::thread_frame_pointers = std::move(job_data->frame_pointers); + DB::Exception::setThreadFramePointers(std::move(job_data->frame_pointers)); CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index c0e8514c62a..e29d2a1e0aa 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -18,6 +18,11 @@ namespace DB static constinit std::atomic allow_logging{true}; +bool OwnSplitChannel::isLoggingEnabled() +{ + return allow_logging; +} + void OwnSplitChannel::disableLogging() { allow_logging = false; @@ -25,7 +30,7 @@ void OwnSplitChannel::disableLogging() void OwnSplitChannel::log(const Poco::Message & msg) { - if (!allow_logging) + if (!isLoggingEnabled()) return; #ifndef WITHOUT_TEXT_LOG diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 9872a4fb558..9de55f330be 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -41,6 +41,8 @@ public: static void disableLogging(); + static bool isLoggingEnabled(); + private: void logSplit(const Poco::Message & msg); void tryLogSplit(const Poco::Message & msg); From c7330252cf581441b95c51b47977f597eb41734e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Jul 2024 11:47:49 +0300 Subject: [PATCH 225/321] Disable convert OUTER JOIN to INNER JOIN optimization for non ALL JOIN strictness --- .../QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp index d90f0e152e7..d9296f10a98 100644 --- a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp @@ -23,7 +23,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan:: return 0; const auto & table_join = join->getJoin()->getTableJoin(); - if (table_join.strictness() == JoinStrictness::Asof) + + /// Any JOIN issue https://github.com/ClickHouse/ClickHouse/issues/66447 + /// Anti JOIN issue https://github.com/ClickHouse/ClickHouse/issues/67156 + if (table_join.strictness() != JoinStrictness::All) return 0; /// TODO: Support join_use_nulls From aaa25454b31d854338200b335d7ac6442e959af4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 26 Jul 2024 10:58:45 +0200 Subject: [PATCH 226/321] Additional formatting fixes --- .../functions/type-conversion-functions.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4326753216e..87d824ec5bb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -571,7 +571,7 @@ Result: ## toInt32OrZero -Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int32](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -767,7 +767,7 @@ SELECT toInt64('9223372036854775808') == --9223372036854775808; **Returned value** -- 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). +- 64-bit integer value. [Int64](../data-types/int-uint.md). :::note The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -800,7 +800,7 @@ Result: ## toInt64OrZero -Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int64](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -878,7 +878,7 @@ Types for which `\N` is returned: **Returned value** -- Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). +- 64-bit integer value if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -935,7 +935,7 @@ Types for which the default value is returned: **Returned value** -- Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). +- 64-bit integer value if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1028,7 +1028,7 @@ Result: ## toInt128OrZero -Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int128](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -1256,7 +1256,7 @@ Result: ## toInt256OrZero -Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int256](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** From 338685cc79a5358246977f2ba039230a615c6ea6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 10:59:17 +0200 Subject: [PATCH 227/321] Fix build --- programs/odbc-bridge/tests/CMakeLists.txt | 2 +- src/CMakeLists.txt | 1 + src/Common/mysqlxx/tests/CMakeLists.txt | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/odbc-bridge/tests/CMakeLists.txt b/programs/odbc-bridge/tests/CMakeLists.txt index f1411dbb554..2f63aed7942 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 clickhouse_common_config) +target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io clickhouse_common_config loggers_no_text_log) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0f84dd35320..fede7d69105 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -638,6 +638,7 @@ if (ENABLE_TESTS) dbms clickhouse_common_config clickhouse_common_zookeeper + loggers hilite_comparator) if (TARGET ch_contrib::simdjson) diff --git a/src/Common/mysqlxx/tests/CMakeLists.txt b/src/Common/mysqlxx/tests/CMakeLists.txt index f62908ddcaf..53bee778470 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 clickhouse_common_config) +target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx clickhouse_common_config loggers_no_text_log) From 8d13461fb74fc991b73382d04a9bc7a9fd3425fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 09:03:37 +0000 Subject: [PATCH 228/321] Another fix. --- src/Storages/StorageDistributed.cpp | 7 ++----- src/Storages/StorageMerge.cpp | 8 ++++++++ 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9b417cda177..07892971ec2 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -847,11 +847,8 @@ void StorageDistributed::read( /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ - if (processed_stage != QueryProcessingStage::Complete) - { - for (auto & column : header) - column.column = column.column->convertToFullColumnIfConst(); - } + for (auto & column : header) + column.column = column.column->convertToFullColumnIfConst(); modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f5bc183931f..9962da3d6de 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -369,6 +369,14 @@ void StorageMerge::read( /// What will be result structure depending on query processed stage in source tables? Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); + if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage == QueryProcessingStage::Complete) + { + /// Remove constants. + /// For StorageDistributed some functions like `hostName` that are constants only for local queries. + for (auto & column : common_header) + column.column = column.column->convertToFullColumnIfConst(); + } + auto step = std::make_unique( column_names, query_info, From 02bfe82192fa4aa6ebb3e7b9192ec6f334fbfc56 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 26 Jul 2024 11:19:46 +0200 Subject: [PATCH 229/321] rename filesystemFree to fiilesystemUnreserved --- docs/en/sql-reference/functions/other-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b7e4094f30e..79bffe00d01 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2102,14 +2102,14 @@ Result: └─────────────────┘ ``` -## filesystemFree +## filesystemUnreserved -Returns the total amount of the free space on the filesystem hosting the database persistence. See also `filesystemAvailable` +Returns the total amount of the free space on the filesystem hosting the database persistence. (previously `filesystemFree`). See also [`filesystemAvailable`](#filesystemavailable). **Syntax** ```sql -filesystemFree() +filesystemUnreserved() ``` **Returned value** @@ -2121,7 +2121,7 @@ filesystemFree() Query: ```sql -SELECT formatReadableSize(filesystemFree()) AS "Free space"; +SELECT formatReadableSize(filesystemUnreserved()) AS "Free space"; ``` Result: From 434571d496a6ca6fc1b0038ead560572d0553ee5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Jul 2024 12:40:20 +0300 Subject: [PATCH 230/321] Added tests --- ...uter_join_to_inner_join_any_join.reference | 3 ++ ...vert_outer_join_to_inner_join_any_join.sql | 33 ++++++++++++++ ...ter_join_to_inner_join_anti_join.reference | 19 ++++++++ ...ert_outer_join_to_inner_join_anti_join.sql | 45 +++++++++++++++++++ 4 files changed, 100 insertions(+) create mode 100644 tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.reference create mode 100644 tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.sql create mode 100644 tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.reference create mode 100644 tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.sql diff --git a/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.reference b/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.reference new file mode 100644 index 00000000000..3d6a23045fb --- /dev/null +++ b/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.reference @@ -0,0 +1,3 @@ +1 tx1 US +1 tx2 US +1 tx3 US diff --git a/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.sql b/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.sql new file mode 100644 index 00000000000..599875e90cf --- /dev/null +++ b/tests/queries/0_stateless/03210_convert_outer_join_to_inner_join_any_join.sql @@ -0,0 +1,33 @@ +DROP TABLE IF EXISTS user_country; +DROP TABLE IF EXISTS user_transactions; + +CREATE TABLE user_country ( + user_id UInt64, + country String +) +ENGINE = ReplacingMergeTree +ORDER BY user_id; + +CREATE TABLE user_transactions ( + user_id UInt64, + transaction_id String +) +ENGINE = MergeTree +ORDER BY user_id; + +INSERT INTO user_country (user_id, country) VALUES (1, 'US'); +INSERT INTO user_transactions (user_id, transaction_id) VALUES (1, 'tx1'), (1, 'tx2'), (1, 'tx3'), (2, 'tx1'); + +-- Expected 3 rows, got only 1. Removing 'ANY' and adding 'FINAL' fixes +-- the issue (but it is not always possible). Moving filter by 'country' to +-- an outer query doesn't help. Query without filter by 'country' works +-- as expected (returns 3 rows). +SELECT * FROM user_transactions +ANY LEFT JOIN user_country USING (user_id) +WHERE + user_id = 1 + AND country = 'US' +ORDER BY ALL; + +DROP TABLE user_country; +DROP TABLE user_transactions; diff --git a/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.reference b/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.reference new file mode 100644 index 00000000000..d717a29ab23 --- /dev/null +++ b/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.reference @@ -0,0 +1,19 @@ +DATA + ┏━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━┓ + ┃ c0 ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━┩ +1. │ 826636805 │ 0 │ │ + ├───────────┼───────────┼────┤ +2. │ 0 │ 150808457 │ │ + └───────────┴───────────┴────┘ +NUMBER OF ROWS IN FIRST SHOULD BE EQUAL TO SECOND +FISRT + +SECOND +1 +TO DEBUG I TOOK JUST A SUBQUERY AND IT HAS 1 ROW +THIRD +1 +AND I ADDED SINGLE CONDITION THAT CONDITION <>0 THAT IS 1 IN THIRD QUERY AND IT HAS NO RESULT!!! +FOURTH +1 diff --git a/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.sql b/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.sql new file mode 100644 index 00000000000..77b1d52dd18 --- /dev/null +++ b/tests/queries/0_stateless/03211_convert_outer_join_to_inner_join_anti_join.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t0; + +CREATE TABLE t0 (c0 Int32, c1 Int32, c2 String) ENGINE = Log() ; +INSERT INTO t0(c0, c1, c2) VALUES (826636805,0, ''), (0, 150808457, ''); + +SELECT 'DATA'; +SELECT * FROM t0 FORMAT PrettyMonoBlock; + +SELECT 'NUMBER OF ROWS IN FIRST SHOULD BE EQUAL TO SECOND'; + + +SELECT 'FISRT'; +SELECT left.c2 FROM t0 AS left +LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1)) +WHERE (abs ((- ((sign (right_0.c1)))))); + +SELECT 'SECOND'; +SELECT SUM(check <> 0) +FROM +( + SELECT (abs ((- ((sign (right_0.c1)))))) AS `check` + FROM t0 AS left + LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1)) +); + + +SELECT 'TO DEBUG I TOOK JUST A SUBQUERY AND IT HAS 1 ROW'; + +SELECT 'THIRD'; + +SELECT (abs ((- ((sign (right_0.c1)))))) AS `check` +FROM t0 AS left +LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1)); + + +SELECT 'AND I ADDED SINGLE CONDITION THAT CONDITION <>0 THAT IS 1 IN THIRD QUERY AND IT HAS NO RESULT!!!'; + + +SELECT 'FOURTH'; +SELECT (abs ((- ((sign (right_0.c1)))))) AS `check` +FROM t0 AS left +LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1)) +WHERE check <> 0; + +DROP TABLE t0; From b3828b038dbcc9c5cf71b99d58f06497c2af3bd6 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 26 Jul 2024 11:49:04 +0200 Subject: [PATCH 231/321] add `filesystemUnreserved` --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8e4e4fafe29..1a324b98ff4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1622,6 +1622,7 @@ filesystem filesystemAvailable filesystemCapacity filesystemFree +filesystemUnreserved filesystems finalizeAggregation fips From ca9bf2c67c8ac16d4fd18f2def6e4d3dfea62971 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 11:53:48 +0200 Subject: [PATCH 232/321] Fix tidy --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 1250e1273b9..7448d73cbbc 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -136,7 +136,7 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); /// If the balancing strategy has an optimal node then it will be the first in the list - bool connected_to_suboptimal_node = node_idx && *node_idx != shuffled_hosts[0].original_index; + bool connected_to_suboptimal_node = node_idx && static_cast(*node_idx) != shuffled_hosts[0].original_index; bool respect_az = args.prefer_local_availability_zone && !args.client_availability_zone.empty(); bool may_benefit_from_reconnecting = respect_az || args.get_priority_load_balancing.hasOptimalNode(); if (connected_to_suboptimal_node && may_benefit_from_reconnecting) From 7f80dab6927316f5c6c56e51ba439d01161f7567 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 26 Jul 2024 12:34:36 +0200 Subject: [PATCH 233/321] CI push --- docker/test/util/process_functional_tests_result.py | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index dbe50eeade0..3da1a8f3674 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -180,7 +180,6 @@ def process_result(result_path, broken_tests): for result in test_results: if result[1] == "FAIL": result[1] = "SERVER_DIED" - test_results.append(["Server died", "FAIL", "0", ""]) elif not success_finish: description = "Tests are not finished, " From 1ebafccc13ea69ba06e2450014fd15d39facdcaa Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 26 Jul 2024 12:42:07 +0200 Subject: [PATCH 234/321] add `joinGetOrNull` --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 7de065cc589..182e1d2cb33 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1862,6 +1862,7 @@ jdbc jemalloc jeprof joinGet +joinGetOrNull json jsonMergePatch jsonasstring From 1225d50508ad0885dca3367b08c15f54c65b02f6 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 26 Jul 2024 11:09:48 +0000 Subject: [PATCH 235/321] Do not count AttachedTable for tables in information schema databases --- src/Databases/DatabasesCommon.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index d2926c64f29..b8e9231f5c6 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -289,9 +289,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n tables.erase(it); table_storage->is_detached = true; - if (!table_storage->isSystemStorage() - && database_name != DatabaseCatalog::SYSTEM_DATABASE - && database_name != DatabaseCatalog::TEMPORARY_DATABASE) + if (!table_storage->isSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name)) { LOG_TEST(log, "Counting detached table {} to database {}", table_name, database_name); CurrentMetrics::sub(getAttachedCounterForStorage(table_storage)); @@ -339,9 +337,7 @@ 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() - && database_name != DatabaseCatalog::SYSTEM_DATABASE - && database_name != DatabaseCatalog::TEMPORARY_DATABASE) + if (!table->isSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name)) { LOG_TEST(log, "Counting attached table {} to database {}", table_name, database_name); CurrentMetrics::add(getAttachedCounterForStorage(table)); From 0cf0437196dfe4ee0f489ecc040b71e42e1f1a22 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 25 Jul 2024 16:36:32 +0200 Subject: [PATCH 236/321] Use separate client context in clickhouse-local --- programs/client/Client.cpp | 29 ++++----- programs/client/Client.h | 1 - programs/local/LocalServer.cpp | 28 +++++--- programs/local/LocalServer.h | 4 +- src/Client/ClientBase.cpp | 113 ++++++++++++++++++--------------- src/Client/ClientBase.h | 6 ++ 6 files changed, 102 insertions(+), 79 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 887c5cb86bc..f2919db0308 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -209,8 +209,8 @@ std::vector Client::loadWarningMessages() {} /* query_parameters */, "" /* query_id */, QueryProcessingStage::Complete, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), false, {}); + &client_context->getSettingsRef(), + &client_context->getClientInfo(), false, {}); while (true) { Packet packet = connection->receivePacket(); @@ -306,9 +306,6 @@ void Client::initialize(Poco::Util::Application & self) if (env_password && !config().has("password")) config().setString("password", env_password); - // global_context->setApplicationType(Context::ApplicationType::CLIENT); - global_context->setQueryParameters(query_parameters); - /// settings and limits could be specified in config file, but passed settings has higher priority for (const auto & setting : global_context->getSettingsRef().allUnchanged()) { @@ -382,7 +379,7 @@ try showWarnings(); /// Set user password complexity rules - auto & access_control = global_context->getAccessControl(); + auto & access_control = client_context->getAccessControl(); access_control.setPasswordComplexityRules(connection->getPasswordComplexityRules()); if (is_interactive && !delayed_interactive) @@ -459,7 +456,7 @@ void Client::connect() << connection_parameters.host << ":" << connection_parameters.port << (!connection_parameters.user.empty() ? " as user " + connection_parameters.user : "") << "." << std::endl; - connection = Connection::createConnection(connection_parameters, global_context); + connection = Connection::createConnection(connection_parameters, client_context); if (max_client_network_bandwidth) { @@ -528,7 +525,7 @@ void Client::connect() } } - if (!global_context->getSettingsRef().use_client_time_zone) + if (!client_context->getSettingsRef().use_client_time_zone) { const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts); if (!time_zone.empty()) @@ -611,7 +608,7 @@ void Client::printChangedSettings() const } }; - print_changes(global_context->getSettingsRef().changes(), "settings"); + print_changes(client_context->getSettingsRef().changes(), "settings"); print_changes(cmd_merge_tree_settings.changes(), "MergeTree settings"); } @@ -709,7 +706,7 @@ bool Client::processWithFuzzing(const String & full_query) { const char * begin = full_query.data(); orig_ast = parseQuery(begin, begin + full_query.size(), - global_context->getSettingsRef(), + client_context->getSettingsRef(), /*allow_multi_statements=*/ true); } catch (const Exception & e) @@ -733,7 +730,7 @@ bool Client::processWithFuzzing(const String & full_query) } // Kusto is not a subject for fuzzing (yet) - if (global_context->getSettingsRef().dialect == DB::Dialect::kusto) + if (client_context->getSettingsRef().dialect == DB::Dialect::kusto) { return true; } @@ -1072,6 +1069,11 @@ void Client::processOptions(const OptionsDescription & options_description, global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::CLIENT); + /// In case of clickhouse-client the `client_context` can be just an alias for the `global_context`. + /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) + client_context = global_context; + initClientContext(); + global_context->setSettings(cmd_settings); /// Copy settings-related program options to config. @@ -1205,11 +1207,6 @@ void Client::processConfig() pager = config().getString("pager", ""); setDefaultFormatsAndCompressionFromConfiguration(); - - global_context->setClientName(std::string(DEFAULT_CLIENT_NAME)); - global_context->setQueryKindInitial(); - global_context->setQuotaClientKey(config().getString("quota_key", "")); - global_context->setQueryKind(query_kind); } diff --git a/programs/client/Client.h b/programs/client/Client.h index 6d57a6ea648..ff71b36dbf3 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -19,7 +19,6 @@ public: int main(const std::vector & /*args*/) override; protected: - Poco::Util::LayeredConfiguration & getClientConfiguration() override; bool processWithFuzzing(const String & full_query) override; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 48e0cca7b73..e60c8ef6085 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -295,6 +295,8 @@ void LocalServer::cleanup() if (suggest) suggest.reset(); + client_context.reset(); + if (global_context) { global_context->shutdown(); @@ -436,7 +438,7 @@ void LocalServer::connect() in = input.get(); } connection = LocalConnection::createConnection( - connection_parameters, global_context, in, need_render_progress, need_render_profile_events, server_display_name); + connection_parameters, client_context, in, need_render_progress, need_render_profile_events, server_display_name); } @@ -497,8 +499,6 @@ try initTTYBuffer(toProgressOption(getClientConfiguration().getString("progress", "default"))); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); - applyCmdSettings(global_context); - /// try to load user defined executable functions, throw on error and die try { @@ -510,6 +510,11 @@ try throw; } + /// Must be called after we stopped initializing the global context and changing its settings. + /// After this point the global context must be stayed almost unchanged till shutdown, + /// and all necessary changes must be made to the client context instead. + createClientContext(); + if (is_interactive) { clearTerminal(); @@ -730,11 +735,12 @@ void LocalServer::processConfig() /// there is separate context for Buffer tables). adjustSettings(); applySettingsOverridesForLocal(global_context); - applyCmdOptions(global_context); /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(getClientConfiguration()); + applyCmdOptions(global_context); + /// We load temporary database first, because projections need it. DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); @@ -778,10 +784,6 @@ void LocalServer::processConfig() server_display_name = getClientConfiguration().getString("display_name", ""); prompt_by_server_display_name = getClientConfiguration().getRawString("prompt_by_server_display_name.default", ":) "); - - global_context->setQueryKindInitial(); - global_context->setQueryKind(query_kind); - global_context->setQueryParameters(query_parameters); } @@ -860,6 +862,16 @@ void LocalServer::applyCmdOptions(ContextMutablePtr context) } +void LocalServer::createClientContext() +{ + /// In case of clickhouse-local it's necessary to use a separate context for client-related purposes. + /// We can't just change the global context because it is used in background tasks (for example, in merges) + /// which don't expect that the global context can suddenly change. + client_context = Context::createCopy(global_context); + initClientContext(); +} + + void LocalServer::processOptions(const OptionsDescription &, const CommandLineOptions & options, const std::vector &, const std::vector &) { if (options.count("table")) diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 0715f358313..ae9980311e1 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -31,7 +31,6 @@ public: int main(const std::vector & /*args*/) override; protected: - Poco::Util::LayeredConfiguration & getClientConfiguration() override; void connect() override; @@ -50,7 +49,6 @@ protected: void processConfig() override; void readArguments(int argc, char ** argv, Arguments & common_arguments, std::vector &, std::vector &) override; - void updateLoggerLevel(const String & logs_level) override; private: @@ -67,6 +65,8 @@ private: void applyCmdOptions(ContextMutablePtr context); void applyCmdSettings(ContextMutablePtr context); + void createClientContext(); + ServerSettings server_settings; std::optional status; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13dce05cabc..50cc6b98b81 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -467,7 +467,7 @@ void ClientBase::sendExternalTables(ASTPtr parsed_query) std::vector data; for (auto & table : external_tables) - data.emplace_back(table.getData(global_context)); + data.emplace_back(table.getData(client_context)); connection->sendExternalTablesData(data); } @@ -680,10 +680,10 @@ try /// intermixed with data with parallel formatting. /// It may increase code complexity significantly. if (!extras_into_stdout || select_only_into_file) - output_format = global_context->getOutputFormatParallelIfPossible( + output_format = client_context->getOutputFormatParallelIfPossible( current_format, out_file_buf ? *out_file_buf : *out_buf, block); else - output_format = global_context->getOutputFormat( + output_format = client_context->getOutputFormat( current_format, out_file_buf ? *out_file_buf : *out_buf, block); output_format->setAutoFlush(); @@ -762,6 +762,15 @@ void ClientBase::adjustSettings() global_context->setSettings(settings); } +void ClientBase::initClientContext() +{ + client_context->setClientName(std::string(DEFAULT_CLIENT_NAME)); + client_context->setQuotaClientKey(getClientConfiguration().getString("quota_key", "")); + client_context->setQueryKindInitial(); + client_context->setQueryKind(query_kind); + client_context->setQueryParameters(query_parameters); +} + bool ClientBase::isRegularFile(int fd) { struct stat file_stat; @@ -952,7 +961,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) /// client-side. Thus we need to parse the query. const char * begin = full_query.data(); auto parsed_query = parseQuery(begin, begin + full_query.size(), - global_context->getSettingsRef(), + client_context->getSettingsRef(), /*allow_multi_statements=*/ false); if (!parsed_query) @@ -975,7 +984,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) /// But for asynchronous inserts we don't extract data, because it's needed /// to be done on server side in that case (for coalescing the data from multiple inserts on server side). const auto * insert = parsed_query->as(); - if (insert && isSyncInsertWithData(*insert, global_context)) + if (insert && isSyncInsertWithData(*insert, client_context)) query_to_execute = full_query.substr(0, insert->data - full_query.data()); else query_to_execute = full_query; @@ -1093,7 +1102,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } } - const auto & settings = global_context->getSettingsRef(); + const auto & settings = client_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; int retries_left = 10; @@ -1108,10 +1117,10 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + client_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &client_context->getSettingsRef(), + &client_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); @@ -1298,7 +1307,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - global_context->setSetting("session_timezone", tz); + client_context->setSetting("session_timezone", tz); } @@ -1494,13 +1503,13 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query) { - if (!global_context->hasInsertionTable() && insert_query.table) + if (!client_context->hasInsertionTable() && insert_query.table) { String table = insert_query.table->as().shortName(); if (!table.empty()) { String database = insert_query.database ? insert_query.database->as().shortName() : ""; - global_context->setInsertionTable(StorageID(database, table)); + client_context->setInsertionTable(StorageID(database, table)); } } } @@ -1551,7 +1560,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars const auto & parsed_insert_query = parsed_query->as(); if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in)))) { - const auto & settings = global_context->getSettingsRef(); + const auto & settings = client_context->getSettingsRef(); if (settings.throw_if_no_data_to_insert) throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert"); else @@ -1565,10 +1574,10 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + client_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &client_context->getSettingsRef(), + &client_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); @@ -1616,7 +1625,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des /// Set callback to be called on file progress. if (tty_buf) - progress_indication.setFileProgressCallback(global_context, *tty_buf); + progress_indication.setFileProgressCallback(client_context, *tty_buf); } /// If data fetched from file (maybe compressed file) @@ -1650,10 +1659,10 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des } StorageFile::CommonArguments args{ - WithContext(global_context), + WithContext(client_context), parsed_insert_query->table_id, current_format, - getFormatSettings(global_context), + getFormatSettings(client_context), compression_method, columns_for_storage_file, ConstraintsDescription{}, @@ -1661,7 +1670,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des {}, String{}, }; - StoragePtr storage = std::make_shared(in_file, global_context->getUserFilesPath(), args); + StoragePtr storage = std::make_shared(in_file, client_context->getUserFilesPath(), args); storage->startup(); SelectQueryInfo query_info; @@ -1672,16 +1681,16 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des storage->read( plan, sample.getNames(), - storage->getStorageSnapshot(metadata, global_context), + storage->getStorageSnapshot(metadata, client_context), query_info, - global_context, + client_context, {}, - global_context->getSettingsRef().max_block_size, + client_context->getSettingsRef().max_block_size, getNumberOfPhysicalCPUCores()); auto builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_context), - BuildQueryPipelineSettings::fromContext(global_context)); + QueryPlanOptimizationSettings::fromContext(client_context), + BuildQueryPipelineSettings::fromContext(client_context)); QueryPlanResourceHolder resources; auto pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources); @@ -1742,14 +1751,14 @@ void ClientBase::sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDes current_format = insert->format; } - auto source = global_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size); + auto source = client_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size); Pipe pipe(source); if (columns_description.hasDefaults()) { pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, columns_description, *source, global_context); + return std::make_shared(header, columns_description, *source, client_context); }); } @@ -1911,12 +1920,12 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (is_interactive) { - global_context->setCurrentQueryId(""); + client_context->setCurrentQueryId(""); // Generate a new query_id for (const auto & query_id_format : query_id_formats) { writeString(query_id_format.first, std_out); - writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", global_context->getCurrentQueryId())), std_out); + writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", client_context->getCurrentQueryId())), std_out); writeChar('\n', std_out); std_out.next(); } @@ -1943,7 +1952,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin auto password = auth_data->getPassword(); if (password) - global_context->getAccessControl().checkPasswordComplexityRules(*password); + client_context->getAccessControl().checkPasswordComplexityRules(*password); } } } @@ -1958,15 +1967,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin std::optional old_settings; SCOPE_EXIT_SAFE({ if (old_settings) - global_context->setSettings(*old_settings); + client_context->setSettings(*old_settings); }); auto apply_query_settings = [&](const IAST & settings_ast) { if (!old_settings) - old_settings.emplace(global_context->getSettingsRef()); - global_context->applySettingsChanges(settings_ast.as()->changes); - global_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); + old_settings.emplace(client_context->getSettingsRef()); + client_context->applySettingsChanges(settings_ast.as()->changes); + client_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); }; const auto * insert = parsed_query->as(); @@ -1999,7 +2008,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (insert && insert->select) insert->tryFindInputFunction(input_function); - bool is_async_insert_with_inlined_data = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + bool is_async_insert_with_inlined_data = client_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); if (is_async_insert_with_inlined_data) { @@ -2034,9 +2043,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (change.name == "profile") current_profile = change.value.safeGet(); else - global_context->applySettingChange(change); + client_context->applySettingChange(change); } - global_context->resetSettingsToDefaultValue(set_query->default_settings); + client_context->resetSettingsToDefaultValue(set_query->default_settings); /// Query parameters inside SET queries should be also saved on the client side /// to override their previous definitions set with --param_* arguments @@ -2044,7 +2053,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin for (const auto & [name, value] : set_query->query_parameters) query_parameters.insert_or_assign(name, value); - global_context->addQueryParameters(NameToNameMap{set_query->query_parameters.begin(), set_query->query_parameters.end()}); + client_context->addQueryParameters(NameToNameMap{set_query->query_parameters.begin(), set_query->query_parameters.end()}); } if (const auto * use_query = parsed_query->as()) { @@ -2121,8 +2130,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( if (this_query_begin >= all_queries_end) return MultiQueryProcessingStage::QUERIES_END; - unsigned max_parser_depth = static_cast(global_context->getSettingsRef().max_parser_depth); - unsigned max_parser_backtracks = static_cast(global_context->getSettingsRef().max_parser_backtracks); + unsigned max_parser_depth = static_cast(client_context->getSettingsRef().max_parser_depth); + unsigned max_parser_backtracks = static_cast(client_context->getSettingsRef().max_parser_backtracks); // If there are only comments left until the end of file, we just // stop. The parser can't handle this situation because it always @@ -2142,7 +2151,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( try { parsed_query = parseQuery(this_query_end, all_queries_end, - global_context->getSettingsRef(), + client_context->getSettingsRef(), /*allow_multi_statements=*/ true); } catch (const Exception & e) @@ -2185,7 +2194,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( { this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); insert_ast->end = this_query_end; - query_to_execute_end = isSyncInsertWithData(*insert_ast, global_context) ? insert_ast->data : this_query_end; + query_to_execute_end = isSyncInsertWithData(*insert_ast, client_context) ? insert_ast->data : this_query_end; } query_to_execute = all_queries_text.substr(this_query_begin - all_queries_text.data(), query_to_execute_end - this_query_begin); @@ -2387,13 +2396,13 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // , where the inline data is delimited by semicolon and not by a // newline. auto * insert_ast = parsed_query->as(); - if (insert_ast && isSyncInsertWithData(*insert_ast, global_context)) + if (insert_ast && isSyncInsertWithData(*insert_ast, client_context)) { this_query_end = insert_ast->end; adjustQueryEnd( this_query_end, all_queries_end, - static_cast(global_context->getSettingsRef().max_parser_depth), - static_cast(global_context->getSettingsRef().max_parser_backtracks)); + static_cast(client_context->getSettingsRef().max_parser_depth), + static_cast(client_context->getSettingsRef().max_parser_backtracks)); } // Report error. @@ -2523,10 +2532,10 @@ void ClientBase::runInteractive() if (load_suggestions) { /// Load suggestion data from the server. - if (global_context->getApplicationType() == Context::ApplicationType::CLIENT) - suggest->load(global_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load); - else if (global_context->getApplicationType() == Context::ApplicationType::LOCAL) - suggest->load(global_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load); + if (client_context->getApplicationType() == Context::ApplicationType::CLIENT) + suggest->load(client_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load); + else if (client_context->getApplicationType() == Context::ApplicationType::LOCAL) + suggest->load(client_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load); } if (home_path.empty()) @@ -2664,7 +2673,7 @@ void ClientBase::runInteractive() { // If a separate connection loading suggestions failed to open a new session, // use the main session to receive them. - suggest->load(*connection, connection_parameters.timeouts, getClientConfiguration().getInt("suggestion_limit"), global_context->getClientInfo()); + suggest->load(*connection, connection_parameters.timeouts, getClientConfiguration().getInt("suggestion_limit"), client_context->getClientInfo()); } try @@ -2713,10 +2722,10 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name) if (!getClientConfiguration().has("log_comment")) { - Settings settings = global_context->getSettings(); + Settings settings = client_context->getSettings(); /// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]" settings.log_comment = fs::absolute(fs::path(file_name)); - global_context->setSettings(settings); + client_context->setSettings(settings); } return executeMultiQuery(queries_from_file); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 4f500a4c45d..be74090b84d 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -206,6 +206,9 @@ protected: /// Adjust some settings after command line options and config had been processed. void adjustSettings(); + /// Initializes the client context. + void initClientContext(); + void setDefaultFormatsAndCompressionFromConfiguration(); void initTTYBuffer(ProgressOption progress); @@ -215,6 +218,9 @@ protected: SharedContextHolder shared_context; ContextMutablePtr global_context; + /// Client context is a context used only by the client to parse queries, process query parameters and to connect to clickhouse-server. + ContextMutablePtr client_context; + LoggerPtr fatal_log; Poco::AutoPtr fatal_channel_ptr; Poco::AutoPtr fatal_console_channel_ptr; From 3cf2ec36ca31964a5a57717d5645f5e5a287dd00 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 11:36:54 +0000 Subject: [PATCH 237/321] Verbose output for 03203_client_benchmark_options --- .../03203_client_benchmark_options.sh | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh index a9b9d69822b..475309cebb9 100755 --- a/tests/queries/0_stateless/03203_client_benchmark_options.sh +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -4,10 +4,23 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" -${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" -${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" -${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" # expected no output -${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" -${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9].*B$" && echo "Ok" || echo "Fail" -${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "BAD_ARGUMENTS" && echo "Ok" || echo "Fail" +output=$(${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) +echo "$output" | grep -q "^2\." && echo "Ok" || { echo "Fail"; echo "$output"; } + +output=$(${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) +echo "$output" | grep -q "^2\." && echo "Ok" || { echo "Fail"; echo "$output"; } + +output=$(${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) +echo "$output" | grep -q "^[0-9]\+$" && echo "Ok" || { echo "Fail"; echo "$output"; } + +output=$(${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) +echo -n "$output" # expected no output + +output=$(${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) +echo "$output" | grep -q "^[0-9]\+$" && echo "Ok" || { echo "Fail"; echo "$output"; } + +output=$(${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) +echo "$output" | grep -q "^[0-9].*B$" && echo "Ok" || { echo "Fail"; echo "$output"; } + +output=$(${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) +echo "$output" | grep -q "BAD_ARGUMENTS" && echo "Ok" || { echo "Fail"; echo "$output"; } From 0299475202b59a4d1a54f13f02b7cc9ff44f38cc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 26 Jul 2024 14:02:37 +0100 Subject: [PATCH 238/321] impl --- ...eplicas_join_algo_and_analyzer_1.reference | 30 +++++ ...allel_replicas_join_algo_and_analyzer_1.sh | 51 ++++++++ ...eplicas_join_algo_and_analyzer_2.reference | 57 +++++++++ ...allel_replicas_join_algo_and_analyzer_2.sh | 103 +++++++++++++++ ...plicas_join_algo_and_analyzer_3.reference} | 87 ------------- ...llel_replicas_join_algo_and_analyzer_3.sh} | 119 ------------------ 6 files changed, 241 insertions(+), 206 deletions(-) create mode 100644 tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.reference create mode 100755 tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh create mode 100644 tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.reference create mode 100755 tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.sh rename tests/queries/0_stateless/{02967_parallel_replicas_join_algo_and_analyzer.reference => 02967_parallel_replicas_join_algo_and_analyzer_3.reference} (55%) rename tests/queries/0_stateless/{02967_parallel_replicas_join_algo_and_analyzer.sh => 02967_parallel_replicas_join_algo_and_analyzer_3.sh} (58%) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.reference new file mode 100644 index 00000000000..e1bf9c27a81 --- /dev/null +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.reference @@ -0,0 +1,30 @@ + +simple join with analyzer +4200000 4200000 4200000 -1400000 +4200006 4200006 4200006 -1400002 +4200012 4200012 4200012 -1400004 +4200018 4200018 4200018 -1400006 +4200024 4200024 4200024 -1400008 +4200030 4200030 4200030 -1400010 +4200036 4200036 4200036 -1400012 +4200042 4200042 4200042 -1400014 +4200048 4200048 4200048 -1400016 +4200054 4200054 4200054 -1400018 + +simple (global) join with analyzer and parallel replicas +4200000 4200000 4200000 -1400000 +4200006 4200006 4200006 -1400002 +4200012 4200012 4200012 -1400004 +4200018 4200018 4200018 -1400006 +4200024 4200024 4200024 -1400008 +4200030 4200030 4200030 -1400010 +4200036 4200036 4200036 -1400012 +4200042 4200042 4200042 -1400014 +4200048 4200048 4200048 -1400016 +4200054 4200054 4200054 -1400018 +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) + DefaultCoordinator: Coordination done +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) + DefaultCoordinator: Coordination done diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh new file mode 100755 index 00000000000..1089eb4051f --- /dev/null +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# Tags: long, no-random-settings, no-random-merge-tree-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -nm -q " +drop table if exists num_1; +drop table if exists num_2; + +create table num_1 (key UInt64, value String) engine = MergeTree order by key; +create table num_2 (key UInt64, value Int64) engine = MergeTree order by key; + +insert into num_1 select number * 2, toString(number * 2) from numbers(1e7); +insert into num_2 select number * 3, -number from numbers(1.5e6); +" + +############## +echo +echo "simple join with analyzer" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1" + +############## +echo +echo "simple (global) join with analyzer and parallel replicas" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, +max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level='trace', +max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +grep "executeQuery\|.*Coordinator: Coordination done" | +grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | +sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.reference new file mode 100644 index 00000000000..297ec311f3e --- /dev/null +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.reference @@ -0,0 +1,57 @@ + +simple (local) join with analyzer and parallel replicas +4200000 4200000 4200000 -1400000 +4200006 4200006 4200006 -1400002 +4200012 4200012 4200012 -1400004 +4200018 4200018 4200018 -1400006 +4200024 4200024 4200024 -1400008 +4200030 4200030 4200030 -1400010 +4200036 4200036 4200036 -1400012 +4200042 4200042 4200042 -1400014 +4200048 4200048 4200048 -1400016 +4200054 4200054 4200054 -1400018 +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) + DefaultCoordinator: Coordination done + +simple (local) join with analyzer and parallel replicas and full sorting merge join +4200000 4200000 4200000 -1400000 +4200006 4200006 4200006 -1400002 +4200012 4200012 4200012 -1400004 +4200018 4200018 4200018 -1400006 +4200024 4200024 4200024 -1400008 +4200030 4200030 4200030 -1400010 +4200036 4200036 4200036 -1400012 +4200042 4200042 4200042 -1400014 +4200048 4200048 4200048 -1400016 +4200054 4200054 4200054 -1400018 +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) + WithOrderCoordinator: Coordination done + +nested join with analyzer +420000 420000 420000 -140000 +420042 420042 420042 -140014 +420084 420084 420084 -140028 +420126 420126 420126 -140042 +420168 420168 420168 -140056 +420210 420210 420210 -140070 +420252 420252 420252 -140084 +420294 420294 420294 -140098 +420336 420336 420336 -140112 +420378 420378 420378 -140126 + +nested join with analyzer and parallel replicas, both local +420000 420000 420000 -140000 +420042 420042 420042 -140014 +420084 420084 420084 -140028 +420126 420126 420126 -140042 +420168 420168 420168 -140056 +420210 420210 420210 -140070 +420252 420252 420252 -140084 +420294 420294 420294 -140098 +420336 420336 420336 -140112 +420378 420378 420378 -140126 +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) +SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) + WithOrderCoordinator: Coordination done diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.sh new file mode 100755 index 00000000000..7a0e2d9bfdb --- /dev/null +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_2.sh @@ -0,0 +1,103 @@ +#!/usr/bin/env bash +# Tags: long, no-random-settings, no-random-merge-tree-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -nm -q " +drop table if exists num_1; +drop table if exists num_2; + +create table num_1 (key UInt64, value String) engine = MergeTree order by key; +create table num_2 (key UInt64, value Int64) engine = MergeTree order by key; + +insert into num_1 select number * 2, toString(number * 2) from numbers(1e7); +insert into num_2 select number * 3, -number from numbers(1.5e6); +" + +############## +echo +echo "simple (local) join with analyzer and parallel replicas" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | +grep "executeQuery\|.*Coordinator: Coordination done" | +grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | +sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' + + +############## +echo +echo "simple (local) join with analyzer and parallel replicas and full sorting merge join" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2) r on l.key = r.key +order by l.key limit 10 offset 700000 +SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | +grep "executeQuery\|.*Coordinator: Coordination done" | +grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | +sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' + + +############## +echo +echo "nested join with analyzer" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2 inner join + (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r +on l.key = r.key order by l.key limit 10 offset 10000 +SETTINGS allow_experimental_analyzer=1" + + +############## +echo +echo "nested join with analyzer and parallel replicas, both local" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2 inner join + (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r +on l.key = r.key order by l.key limit 10 offset 10000 +SETTINGS allow_experimental_analyzer=1, +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2 inner join + (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r +on l.key = r.key order by l.key limit 10 offset 10000 +SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, +cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | +grep "executeQuery\|.*Coordinator: Coordination done" | +grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | +sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference similarity index 55% rename from tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference rename to tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference index d7fa419aeab..c0485b817c4 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference @@ -1,91 +1,4 @@ -simple join with analyzer -4200000 4200000 4200000 -1400000 -4200006 4200006 4200006 -1400002 -4200012 4200012 4200012 -1400004 -4200018 4200018 4200018 -1400006 -4200024 4200024 4200024 -1400008 -4200030 4200030 4200030 -1400010 -4200036 4200036 4200036 -1400012 -4200042 4200042 4200042 -1400014 -4200048 4200048 4200048 -1400016 -4200054 4200054 4200054 -1400018 - -simple (global) join with analyzer and parallel replicas -4200000 4200000 4200000 -1400000 -4200006 4200006 4200006 -1400002 -4200012 4200012 4200012 -1400004 -4200018 4200018 4200018 -1400006 -4200024 4200024 4200024 -1400008 -4200030 4200030 4200030 -1400010 -4200036 4200036 4200036 -1400012 -4200042 4200042 4200042 -1400014 -4200048 4200048 4200048 -1400016 -4200054 4200054 4200054 -1400018 -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) - DefaultCoordinator: Coordination done -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) - DefaultCoordinator: Coordination done - -simple (local) join with analyzer and parallel replicas -4200000 4200000 4200000 -1400000 -4200006 4200006 4200006 -1400002 -4200012 4200012 4200012 -1400004 -4200018 4200018 4200018 -1400006 -4200024 4200024 4200024 -1400008 -4200030 4200030 4200030 -1400010 -4200036 4200036 4200036 -1400012 -4200042 4200042 4200042 -1400014 -4200048 4200048 4200048 -1400016 -4200054 4200054 4200054 -1400018 -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) - DefaultCoordinator: Coordination done - -simple (local) join with analyzer and parallel replicas and full sorting merge join -4200000 4200000 4200000 -1400000 -4200006 4200006 4200006 -1400002 -4200012 4200012 4200012 -1400004 -4200018 4200018 4200018 -1400006 -4200024 4200024 4200024 -1400008 -4200030 4200030 4200030 -1400010 -4200036 4200036 4200036 -1400012 -4200042 4200042 4200042 -1400014 -4200048 4200048 4200048 -1400016 -4200054 4200054 4200054 -1400018 -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) - WithOrderCoordinator: Coordination done - -nested join with analyzer -420000 420000 420000 -140000 -420042 420042 420042 -140014 -420084 420084 420084 -140028 -420126 420126 420126 -140042 -420168 420168 420168 -140056 -420210 420210 420210 -140070 -420252 420252 420252 -140084 -420294 420294 420294 -140098 -420336 420336 420336 -140112 -420378 420378 420378 -140126 - -nested join with analyzer and parallel replicas, both local -420000 420000 420000 -140000 -420042 420042 420042 -140014 -420084 420084 420084 -140028 -420126 420126 420126 -140042 -420168 420168 420168 -140056 -420210 420210 420210 -140070 -420252 420252 420252 -140084 -420294 420294 420294 -140098 -420336 420336 420336 -140112 -420378 420378 420378 -140126 -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) - WithOrderCoordinator: Coordination done - nested join with analyzer and parallel replicas, both global 420000 420000 420000 -140000 420042 420042 420042 -140014 diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh similarity index 58% rename from tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh rename to tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh index 2840482da6d..e49a340ab67 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh @@ -17,125 +17,6 @@ insert into num_1 select number * 2, toString(number * 2) from numbers(1e7); insert into num_2 select number * 3, -number from numbers(1.5e6); " -############## -echo -echo "simple join with analyzer" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1" - -############## -echo -echo "simple (global) join with analyzer and parallel replicas" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level='trace', -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | -grep "executeQuery\|.*Coordinator: Coordination done" | -grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | -sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' - -############## -echo -echo "simple (local) join with analyzer and parallel replicas" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | -grep "executeQuery\|.*Coordinator: Coordination done" | -grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | -sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' - - -############## -echo -echo "simple (local) join with analyzer and parallel replicas and full sorting merge join" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2) r on l.key = r.key -order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | -grep "executeQuery\|.*Coordinator: Coordination done" | -grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | -sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' - - -############## -echo -echo "nested join with analyzer" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2 inner join - (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r -on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1" - - -############## -echo -echo "nested join with analyzer and parallel replicas, both local" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2 inner join - (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r -on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2 inner join - (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r -on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | -grep "executeQuery\|.*Coordinator: Coordination done" | -grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | -sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' - - ############## echo echo "nested join with analyzer and parallel replicas, both global" From d2b3be2fb8345436422e6214f7652545696be6ea Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 15:05:03 +0200 Subject: [PATCH 239/321] Apply optimizations for a single file --- src/CMakeLists.txt | 3 + src/Client/ClientBase.cpp | 163 ---------------------- src/Client/ClientBaseOptimizedParts.cpp | 178 ++++++++++++++++++++++++ 3 files changed, 181 insertions(+), 163 deletions(-) create mode 100644 src/Client/ClientBaseOptimizedParts.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0f84dd35320..8c133971785 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -226,6 +226,9 @@ add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_storages_s3queue Storages/ObjectStorageQueue) add_object_library(clickhouse_storages_materializedview Storages/MaterializedView) add_object_library(clickhouse_client Client) +# Always compile this file with the highest possible level of optimizations, even in Debug builds. +# https://github.com/ClickHouse/ClickHouse/issues/65745 +set_source_files_properties(Client/ClientBaseOptimizedParts.cpp PROPERTIES COMPILE_FLAGS "-O3") add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) add_object_library(clickhouse_server_http Server/HTTP) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 1e1917e1ca1..04af9db7afe 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -108,7 +108,6 @@ namespace ErrorCodes extern const int UNEXPECTED_PACKET_FROM_SERVER; extern const int INVALID_USAGE_OF_INPUT; extern const int CANNOT_SET_SIGNAL_HANDLER; - extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; extern const int FILE_ALREADY_EXISTS; @@ -2848,168 +2847,6 @@ void ClientBase::showClientVersion() output_stream << VERSION_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; } -namespace -{ - -/// Define transparent hash to we can use -/// std::string_view with the containers -struct TransparentStringHash -{ - using is_transparent = void; - size_t operator()(std::string_view txt) const - { - return std::hash{}(txt); - } -}; - -/* - * This functor is used to parse command line arguments and replace dashes with underscores, - * allowing options to be specified using either dashes or underscores. - */ -class OptionsAliasParser -{ -public: - explicit OptionsAliasParser(const boost::program_options::options_description& options) - { - options_names.reserve(options.options().size()); - for (const auto& option : options.options()) - options_names.insert(option->long_name()); - } - - /* - * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options - * Implements boost::program_options::ext_parser logic - */ - std::pair operator()(const std::string & token) const - { - if (!token.starts_with("--")) - return {}; - std::string arg = token.substr(2); - - // divide token by '=' to separate key and value if options style=long_allow_adjacent - auto pos_eq = arg.find('='); - std::string key = arg.substr(0, pos_eq); - - if (options_names.contains(key)) - // option does not require any changes, because it is already correct - return {}; - - std::replace(key.begin(), key.end(), '-', '_'); - if (!options_names.contains(key)) - // after replacing '-' with '_' argument is still unknown - return {}; - - std::string value; - if (pos_eq != std::string::npos && pos_eq < arg.size()) - value = arg.substr(pos_eq + 1); - - return {key, value}; - } - -private: - std::unordered_set options_names; -}; - -} - -/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests -#if defined(__clang__) -#pragma clang optimize on -#endif -void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) -{ - if (allow_repeated_settings) - addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); - else - addProgramOptions(cmd_settings, options_description.main_description.value()); - - if (allow_merge_tree_settings) - { - /// Add merge tree settings manually, because names of some settings - /// may clash. Query settings have higher priority and we just - /// skip ambiguous merge tree settings. - auto & main_options = options_description.main_description.value(); - - std::unordered_set> main_option_names; - for (const auto & option : main_options.options()) - main_option_names.insert(option->long_name()); - - for (const auto & setting : cmd_merge_tree_settings.all()) - { - const auto add_setting = [&](const std::string_view name) - { - if (auto it = main_option_names.find(name); it != main_option_names.end()) - return; - - if (allow_repeated_settings) - addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); - else - addProgramOption(cmd_merge_tree_settings, main_options, name, setting); - }; - - const auto & setting_name = setting.getName(); - - add_setting(setting_name); - - const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); - if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) - { - for (const auto alias : it->second) - { - add_setting(alias); - } - } - } - } - - /// Parse main commandline options. - auto parser = po::command_line_parser(arguments) - .options(options_description.main_description.value()) - .extra_parser(OptionsAliasParser(options_description.main_description.value())) - .allow_unregistered(); - po::parsed_options parsed = parser.run(); - - /// Check unrecognized options without positional options. - auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); - if (!unrecognized_options.empty()) - { - auto hints = this->getHints(unrecognized_options[0]); - if (!hints.empty()) - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", - unrecognized_options[0], toString(hints)); - - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); - } - - /// Check positional options. - for (const auto & op : parsed.options) - { - if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") - && !op.original_tokens[0].empty() && !op.value.empty()) - { - /// Two special cases for better usability: - /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" - /// These are relevant for interactive usage - user-friendly, but questionable in general. - /// In case of ambiguity or for scripts, prefer using proper options. - - const auto & token = op.original_tokens[0]; - po::variable_value value(boost::any(op.value), false); - - const char * option; - if (token.contains(' ')) - option = "query"; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - - if (!options.emplace(option, value).second) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - } - } - - po::store(parsed, options); -} - - void ClientBase::init(int argc, char ** argv) { namespace po = boost::program_options; diff --git a/src/Client/ClientBaseOptimizedParts.cpp b/src/Client/ClientBaseOptimizedParts.cpp new file mode 100644 index 00000000000..31614d301b6 --- /dev/null +++ b/src/Client/ClientBaseOptimizedParts.cpp @@ -0,0 +1,178 @@ +#include + +#include + +namespace DB +{ + +/** + * Program ptions parsing is very slow in debug builds and it affects .sh tests + * causing them to timeout sporadically. + * It seems impossible to enable optimizations for a single function (only to disable them), so + * instead we extract the code to a separate source file and compile it with different options. + */ + +/// +namespace ErrorCodes +{ + extern const int UNRECOGNIZED_ARGUMENTS; +} + +namespace +{ + +/// Define transparent hash to we can use +/// std::string_view with the containers +struct TransparentStringHash +{ + using is_transparent = void; + size_t operator()(std::string_view txt) const + { + return std::hash{}(txt); + } +}; + +/* + * This functor is used to parse command line arguments and replace dashes with underscores, + * allowing options to be specified using either dashes or underscores. + */ +class OptionsAliasParser +{ +public: + explicit OptionsAliasParser(const boost::program_options::options_description& options) + { + options_names.reserve(options.options().size()); + for (const auto& option : options.options()) + options_names.insert(option->long_name()); + } + + /* + * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options + * Implements boost::program_options::ext_parser logic + */ + std::pair operator()(const std::string & token) const + { + if (!token.starts_with("--")) + return {}; + std::string arg = token.substr(2); + + // divide token by '=' to separate key and value if options style=long_allow_adjacent + auto pos_eq = arg.find('='); + std::string key = arg.substr(0, pos_eq); + + if (options_names.contains(key)) + // option does not require any changes, because it is already correct + return {}; + + std::replace(key.begin(), key.end(), '-', '_'); + if (!options_names.contains(key)) + // after replacing '-' with '_' argument is still unknown + return {}; + + std::string value; + if (pos_eq != std::string::npos && pos_eq < arg.size()) + value = arg.substr(pos_eq + 1); + + return {key, value}; + } + +private: + std::unordered_set options_names; +}; + +} + +void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +{ + if (allow_repeated_settings) + addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); + else + addProgramOptions(cmd_settings, options_description.main_description.value()); + + if (allow_merge_tree_settings) + { + /// Add merge tree settings manually, because names of some settings + /// may clash. Query settings have higher priority and we just + /// skip ambiguous merge tree settings. + auto & main_options = options_description.main_description.value(); + + std::unordered_set> main_option_names; + for (const auto & option : main_options.options()) + main_option_names.insert(option->long_name()); + + for (const auto & setting : cmd_merge_tree_settings.all()) + { + const auto add_setting = [&](const std::string_view name) + { + if (auto it = main_option_names.find(name); it != main_option_names.end()) + return; + + if (allow_repeated_settings) + addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); + else + addProgramOption(cmd_merge_tree_settings, main_options, name, setting); + }; + + const auto & setting_name = setting.getName(); + + add_setting(setting_name); + + const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); + if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) + { + for (const auto alias : it->second) + { + add_setting(alias); + } + } + } + } + + /// Parse main commandline options. + auto parser = po::command_line_parser(arguments) + .options(options_description.main_description.value()) + .extra_parser(OptionsAliasParser(options_description.main_description.value())) + .allow_unregistered(); + po::parsed_options parsed = parser.run(); + + /// Check unrecognized options without positional options. + auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); + if (!unrecognized_options.empty()) + { + auto hints = this->getHints(unrecognized_options[0]); + if (!hints.empty()) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", + unrecognized_options[0], toString(hints)); + + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); + } + + /// Check positional options. + for (const auto & op : parsed.options) + { + if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") + && !op.original_tokens[0].empty() && !op.value.empty()) + { + /// Two special cases for better usability: + /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" + /// These are relevant for interactive usage - user-friendly, but questionable in general. + /// In case of ambiguity or for scripts, prefer using proper options. + + const auto & token = op.original_tokens[0]; + po::variable_value value(boost::any(op.value), false); + + const char * option; + if (token.contains(' ')) + option = "query"; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + + if (!options.emplace(option, value).second) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + } + } + + po::store(parsed, options); +} + +} From 031b435e3ad35a57d82ff98ad4e6f79d47d1cbc3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 15:21:11 +0200 Subject: [PATCH 240/321] Style --- src/Client/ClientBaseOptimizedParts.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBaseOptimizedParts.cpp b/src/Client/ClientBaseOptimizedParts.cpp index 31614d301b6..421843a0e79 100644 --- a/src/Client/ClientBaseOptimizedParts.cpp +++ b/src/Client/ClientBaseOptimizedParts.cpp @@ -1,20 +1,18 @@ #include - #include namespace DB { /** - * Program ptions parsing is very slow in debug builds and it affects .sh tests + * Program options parsing is very slow in debug builds and it affects .sh tests * causing them to timeout sporadically. * It seems impossible to enable optimizations for a single function (only to disable them), so * instead we extract the code to a separate source file and compile it with different options. */ - -/// namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int UNRECOGNIZED_ARGUMENTS; } From 414ebf035d9e2f47c16ee93d7ff0d21fbee89bff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 15:32:05 +0200 Subject: [PATCH 241/321] Fix error --- src/IO/ReadWriteBufferFromHTTP.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index a62f22d4bd9..4b2e6580f9b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -140,6 +140,10 @@ std::optional ReadWriteBufferFromHTTP::tryGetFileSize() { return std::nullopt; } + catch (const Poco::IOException &) + { + return std::nullopt; + } } return file_info->file_size; @@ -324,12 +328,12 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, error_message = e.displayText(); exception = std::current_exception(); } - catch (DB::NetException & e) + catch (NetException & e) { error_message = e.displayText(); exception = std::current_exception(); } - catch (DB::HTTPException & e) + catch (HTTPException & e) { if (!isRetriableError(e.getHTTPStatus())) is_retriable = false; @@ -337,7 +341,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, error_message = e.displayText(); exception = std::current_exception(); } - catch (DB::Exception & e) + catch (Exception & e) { is_retriable = false; @@ -708,6 +712,10 @@ std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { return std::nullopt; } + catch (const Poco::IOException &) + { + return std::nullopt; + } } return file_info->last_modified; From 503dc25d1021eb1b598ac52efc0370cfd15c57c6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 15:53:03 +0200 Subject: [PATCH 242/321] Fix linking --- programs/odbc-bridge/tests/CMakeLists.txt | 2 +- src/CMakeLists.txt | 1 - src/Common/Exception.cpp | 6 +++--- src/Common/Logger.cpp | 12 ++++++++++++ src/Common/Logger.h | 4 ++++ src/Daemon/BaseDaemon.cpp | 2 +- src/Loggers/OwnSplitChannel.cpp | 12 ------------ src/Loggers/OwnSplitChannel.h | 4 ---- 8 files changed, 21 insertions(+), 22 deletions(-) diff --git a/programs/odbc-bridge/tests/CMakeLists.txt b/programs/odbc-bridge/tests/CMakeLists.txt index 2f63aed7942..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 clickhouse_common_config loggers_no_text_log) +target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io clickhouse_common_config) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fede7d69105..0f84dd35320 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -638,7 +638,6 @@ if (ENABLE_TESTS) dbms clickhouse_common_config clickhouse_common_zookeeper - loggers hilite_comparator) if (TARGET ch_contrib::simdjson) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index c4bd4fbd943..d68537513da 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -3,12 +3,12 @@ #include #include #include -#include #include #include #include #include #include +#include #include #include #include @@ -253,7 +253,7 @@ void Exception::setThreadFramePointers(ThreadFramePointersBase frame_pointers) static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message) { - if (!OwnSplitChannel::isLoggingEnabled()) + if (!isLoggingEnabled()) return; try @@ -271,7 +271,7 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { - if (!OwnSplitChannel::isLoggingEnabled()) + if (!isLoggingEnabled()) return; /// Under high memory pressure, new allocations throw a diff --git a/src/Common/Logger.cpp b/src/Common/Logger.cpp index c8d557bc3a3..bd848abe353 100644 --- a/src/Common/Logger.cpp +++ b/src/Common/Logger.cpp @@ -25,3 +25,15 @@ bool hasLogger(const std::string & name) { return Poco::Logger::has(name); } + +static constinit std::atomic allow_logging{true}; + +bool isLoggingEnabled() +{ + return allow_logging; +} + +void disableLogging() +{ + allow_logging = false; +} diff --git a/src/Common/Logger.h b/src/Common/Logger.h index b54ccd33e72..7471e3dff9b 100644 --- a/src/Common/Logger.h +++ b/src/Common/Logger.h @@ -64,3 +64,7 @@ LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel, * Otherwise, returns false. */ bool hasLogger(const std::string & name); + +void disableLogging(); + +bool isLoggingEnabled(); diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 366aad00376..e7ae8ea5a1d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -158,7 +158,7 @@ BaseDaemon::~BaseDaemon() tryLogCurrentException(&logger()); } - OwnSplitChannel::disableLogging(); + disableLogging(); } diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index e29d2a1e0aa..c1594361b2c 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -16,18 +16,6 @@ namespace DB { -static constinit std::atomic allow_logging{true}; - -bool OwnSplitChannel::isLoggingEnabled() -{ - return allow_logging; -} - -void OwnSplitChannel::disableLogging() -{ - allow_logging = false; -} - void OwnSplitChannel::log(const Poco::Message & msg) { if (!isLoggingEnabled()) diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 9de55f330be..88bb6b9ce76 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -39,10 +39,6 @@ public: void setLevel(const std::string & name, int level); - static void disableLogging(); - - static bool isLoggingEnabled(); - private: void logSplit(const Poco::Message & msg); void tryLogSplit(const Poco::Message & msg); From 1e12ac577a4ed4f64d4de4feb8110cd794d4ce90 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 26 Jul 2024 14:26:37 +0000 Subject: [PATCH 243/321] Fix flaky `test_pkill_query_log` (tsan) --- tests/integration/test_crash_log/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_crash_log/test.py b/tests/integration/test_crash_log/test.py index a5b82039a84..5a63e6ca6a7 100644 --- a/tests/integration/test_crash_log/test.py +++ b/tests/integration/test_crash_log/test.py @@ -60,6 +60,13 @@ def test_pkill(started_node): def test_pkill_query_log(started_node): + if ( + started_node.is_built_with_thread_sanitizer() + or started_node.is_built_with_address_sanitizer() + or started_node.is_built_with_memory_sanitizer() + ): + pytest.skip("doesn't fit in timeouts for stacktrace generation") + for signal in ["SEGV", "4"]: # force create query_log if it was not created started_node.query("SYSTEM FLUSH LOGS") From d8318fc428e2f5b847415886782fd8e25bca401b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 26 Jul 2024 17:09:22 +0200 Subject: [PATCH 244/321] Wrap in retries --- ...1676_clickhouse_client_autocomplete.python | 38 +++++++++++++++---- 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python index 0f35d259c7c..fe08a07c214 100644 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python @@ -10,18 +10,36 @@ DEBUG_LOG = os.path.join( os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog", ) +STATE_MAP = { + -1: "process did not start", + 0: "completion was found", + 1: "process started and said ':)'", + 2: "completion search was started", + 3: "completion is missing", +} + def run_with_timeout(func, args, timeout): - process = multiprocessing.Process(target=func, args=args) - process.start() - process.join(timeout) + for _ in range(5): + state = multiprocessing.Value("i", -1) + process = multiprocessing.Process(target=func, args=args, kwargs={"state": state}) + process.start() + process.join(timeout) - if process.is_alive(): - process.terminate() - print("Timeout") + if state.value in (0, 3): + return + + if process.is_alive(): + process.terminate() + + if state.value == -1: + continue + + print(f"Timeout, state: {STATE_MAP[state.value]}") + return -def test_completion(program, argv, comp_word): +def test_completion(program, argv, comp_word, state=None): comp_begin = comp_word[:-3] shell_pid, master = pty.fork() @@ -41,6 +59,8 @@ def test_completion(program, argv, comp_word): debug_log_fd.write(repr(output_b) + "\n") debug_log_fd.flush() + state.value = 1 + os.write(master, b"SET " + bytes(comp_begin.encode())) output_b = os.read(master, 4096) output = output_b.decode() @@ -55,6 +75,8 @@ def test_completion(program, argv, comp_word): time.sleep(0.01) os.write(master, b"\t") + state.value = 2 + output_b = os.read(master, 4096) output = output_b.decode() debug_log_fd.write(repr(output_b) + "\n") @@ -65,6 +87,7 @@ def test_completion(program, argv, comp_word): # meaning no concise completion is found if "\x07" in output: print(f"{comp_word}: FAIL") + state.value = 3 return output_b = os.read(master, 4096) @@ -73,6 +96,7 @@ def test_completion(program, argv, comp_word): debug_log_fd.flush() print(f"{comp_word}: OK") + state.value = 0 finally: os.close(master) debug_log_fd.close() From a59036e5152aac2d44b07e0f62ab0ae1a066bb5b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 15:36:15 +0000 Subject: [PATCH 245/321] chmod +x ./tests/queries/0_stateless/03204_format_join_on.sh --- tests/queries/0_stateless/03204_format_join_on.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03204_format_join_on.sh diff --git a/tests/queries/0_stateless/03204_format_join_on.sh b/tests/queries/0_stateless/03204_format_join_on.sh old mode 100644 new mode 100755 From d42fa0690d1b6ec19755b64740d83327e71a914a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 15:59:23 +0000 Subject: [PATCH 246/321] Remove filterBlockWithDAG. --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/System/StorageSystemDroppedTablesParts.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 4 ++-- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 7 +++---- src/Storages/VirtualColumnUtils.h | 2 +- 7 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ecd25e3cf71..d9ab2894dc4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1164,7 +1164,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (valid) { virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_dag), virtual_columns_block, local_context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_dag), local_context), virtual_columns_block); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a37dbfa554c..a6ef0063069 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -488,7 +488,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar return {}; auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), virtual_columns_block, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*dag), context), virtual_columns_block); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index defc4ec2d2a..c2601b8ebe3 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -75,7 +75,7 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(std::optional f { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter), context), block_to_filter); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index a0c9a5c61bd..7ace8ee24aa 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -124,7 +124,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab /// Filter block_to_filter with column 'database'. if (filter_by_database) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_database), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_by_database), context), block_to_filter); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -204,7 +204,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter_by_other_columns) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_other_columns), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_by_other_columns), context), block_to_filter); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 9ae21ded9ba..943ce9c317a 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -125,7 +125,7 @@ ColumnPtr getFilteredTables( block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*dag), context), block); return block.getByPosition(0).column; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index a25b7b5ca49..90c2c7f93c1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,11 +77,10 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context) } } -void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context) +ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(std::move(dag)); - filterBlockWithExpression(actions, block); + return std::make_shared(std::move(dag)); } void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block) @@ -384,7 +383,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, { auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); if (dag) - filterBlockWithDAG(std::move(*dag), block, context); + filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 640f9db2fb8..73b7908b75c 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,7 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context); +ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context); void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block); /// Builds sets used by ActionsDAG inplace. From 454353215736a4c6da635e777b571be0f1bd1831 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 18:33:48 +0200 Subject: [PATCH 247/321] Fix ShellCheck --- .../0_stateless/03203_client_benchmark_options.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh index 475309cebb9..cbbd8aab382 100755 --- a/tests/queries/0_stateless/03203_client_benchmark_options.sh +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -5,22 +5,22 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh output=$(${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -echo "$output" | grep -q "^2\." && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "$output"; } output=$(${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -echo "$output" | grep -q "^2\." && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "$output"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -echo "$output" | grep -q "^[0-9]\+$" && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) echo -n "$output" # expected no output output=$(${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -echo "$output" | grep -q "^[0-9]\+$" && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -echo "$output" | grep -q "^[0-9].*B$" && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9].*B$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -echo "$output" | grep -q "BAD_ARGUMENTS" && echo "Ok" || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "BAD_ARGUMENTS" && echo "Ok"; } || { echo "Fail"; echo "$output"; } From d153a1cf93e157acb7fadb5ca8b4f30fbd08bad5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 18:37:30 +0200 Subject: [PATCH 248/321] add quotes --- .../0_stateless/03203_client_benchmark_options.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh index cbbd8aab382..37a1f2cd3ac 100755 --- a/tests/queries/0_stateless/03203_client_benchmark_options.sh +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -5,22 +5,22 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh output=$(${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) echo -n "$output" # expected no output output=$(${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^[0-9].*B$" && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "^[0-9].*B$" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "BAD_ARGUMENTS" && echo "Ok"; } || { echo "Fail"; echo "$output"; } +{ echo "$output" | grep -q "BAD_ARGUMENTS" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } From f276be829bebd8e704e33565127034f3e258cc31 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 26 Jul 2024 16:59:41 +0000 Subject: [PATCH 249/321] Automatic style fix --- .../0_stateless/01676_clickhouse_client_autocomplete.python | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python index fe08a07c214..f363cb64018 100644 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python @@ -22,7 +22,9 @@ STATE_MAP = { def run_with_timeout(func, args, timeout): for _ in range(5): state = multiprocessing.Value("i", -1) - process = multiprocessing.Process(target=func, args=args, kwargs={"state": state}) + process = multiprocessing.Process( + target=func, args=args, kwargs={"state": state} + ) process.start() process.join(timeout) From 5be75ee349139acb3b70b9b897060c34c170a8cf Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 26 Jul 2024 19:43:54 +0000 Subject: [PATCH 250/321] Fix is done only for the new analyzer --- .../0_stateless/03209_parallel_replicas_order_by_all.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql index 46a3ab4d171..eb3cddbbe07 100644 --- a/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql +++ b/tests/queries/0_stateless/03209_parallel_replicas_order_by_all.sql @@ -10,7 +10,9 @@ ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_03210', 'r1') ORDER BY INSERT INTO order_by_all VALUES ('B', 3, 10), ('C', NULL, 40), ('D', 1, 20), ('A', 2, 30); SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='parallel_replicas'; +SET allow_experimental_analyzer=1; -- fix has been done only for the analyzer SET enable_order_by_all = 0; + -- { echoOn } SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=0; SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = 0, allow_experimental_parallel_reading_from_replicas=1; From 0f4274ac9659efda64e9f362fc74383c0853ba24 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 26 Jul 2024 18:39:38 -0400 Subject: [PATCH 251/321] increase filesystem_cache_reserve_space_wait_lock_timeout_milliseconds to 2000 --- .../00180_no_seek_avoiding_when_reading_from_cache.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh index 2e1b807c496..f9fea2c1dad 100755 --- a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh +++ b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh @@ -17,11 +17,11 @@ $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE hits_s3_sampled FINAL" $CLICKHOUSE_CLIENT -q "SYSTEM DROP FILESYSTEM CACHE" # Warm up the cache -$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" -$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" +$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000" +$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000" query_id=02906_read_from_cache_$RANDOM -$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" +$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000" $CLICKHOUSE_CLIENT -nq " SYSTEM FLUSH LOGS; From 04775ec4fb1375ac1aa7c650233a3e03d44a59bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 02:52:34 +0200 Subject: [PATCH 252/321] English --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1fe2554988..51fe5ee6ec2 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1740,7 +1740,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu const auto * tuple_data_type = typeid_cast(result_type.get()); if (!tuple_data_type) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Qualified matcher {} find non compound expression {} with type {}. Expected tuple or array of tuples. In scope {}", + "Qualified matcher {} found a non-compound expression {} with type {}. Expected a tuple or an array of tuples. In scope {}", matcher_node->formatASTForErrorMessage(), expression_query_tree_node->formatASTForErrorMessage(), expression_query_tree_node->getResultType()->getName(), From f187163fa652d59abf75b8e8bbf1cdb85efffb92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 02:58:00 +0200 Subject: [PATCH 253/321] Fix English --- src/Interpreters/SubstituteColumnOptimizer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SubstituteColumnOptimizer.h b/src/Interpreters/SubstituteColumnOptimizer.h index 28aa8be0801..ecb65cd7707 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.h +++ b/src/Interpreters/SubstituteColumnOptimizer.h @@ -15,7 +15,7 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; /// Optimizer that tries to replace columns to equal columns (according to constraints) -/// with lower size (according to compressed and uncomressed size). +/// with lower size (according to compressed and uncompressed sizes). class SubstituteColumnOptimizer { public: From 0ed2c7e4a00d447c99823aa1b707e392de18c2db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:16:30 +0200 Subject: [PATCH 254/321] Sync with private --- src/IO/ReadBufferFromEmptyFile.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/ReadBufferFromEmptyFile.h b/src/IO/ReadBufferFromEmptyFile.h index b15299dafee..7808ef62fd9 100644 --- a/src/IO/ReadBufferFromEmptyFile.h +++ b/src/IO/ReadBufferFromEmptyFile.h @@ -20,6 +20,7 @@ private: off_t seek(off_t /*off*/, int /*whence*/) override { return 0; } off_t getPosition() override { return 0; } std::optional tryGetFileSize() override { return 0; } + size_t getFileOffsetOfBufferEnd() const override { return 0; } }; } From 90605127c248ec2995c84045fd6a443bff772903 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:35:17 +0200 Subject: [PATCH 255/321] Better exception message --- src/Parsers/IAST.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index ee70fed0f07..4f8edac8597 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -66,7 +66,7 @@ public: /** Set the alias. */ virtual void setAlias(const String & /*to*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't set alias of {}", getColumnName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't set alias of {} of {}", getColumnName(), getID()); } /** Get the text that identifies this element. */ From 10dc9232a11d9733965b508dbacb84c4df6f6637 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:35:25 +0200 Subject: [PATCH 256/321] Remove strange code --- src/Interpreters/SubstituteColumnOptimizer.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index c4aef89fed2..1a7929c1857 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -237,17 +237,6 @@ void SubstituteColumnOptimizer::perform() const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); - // Fill aliases - if (select_query->select()) - { - auto * list = select_query->refSelect()->as(); - if (!list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "List of selected columns must be ASTExpressionList"); - - for (ASTPtr & ast : list->children) - ast->setAlias(ast->getAliasOrColumnName()); - } - auto run_for_all = [&](const auto func) { if (select_query->where()) From 4174726d0dd8e450a2ffd009c95c1d39d2de7060 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:38:31 +0200 Subject: [PATCH 257/321] Add a test --- ...ptimize_with_constraints_logical_error.reference | 0 ...3212_optimize_with_constraints_logical_error.sql | 13 +++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.reference create mode 100644 tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.sql diff --git a/tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.reference b/tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.sql b/tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.sql new file mode 100644 index 00000000000..16a27af986b --- /dev/null +++ b/tests/queries/0_stateless/03212_optimize_with_constraints_logical_error.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +EXPLAIN SYNTAX +WITH 1 AS compound_value SELECT * APPLY (x -> compound_value.*) +FROM test_table WHERE x > 0 +SETTINGS convert_query_to_cnf = true, optimize_using_constraints = true, optimize_substitute_columns = true; -- { serverError UNKNOWN_IDENTIFIER } + +DROP TABLE test_table; From 9969026e4636f1e94abc61816f355ed5d43a1fce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:39:01 +0200 Subject: [PATCH 258/321] Further enhancement --- src/Interpreters/SubstituteColumnOptimizer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index 1a7929c1857..ec51db56f14 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -237,6 +237,9 @@ void SubstituteColumnOptimizer::perform() const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); + if (compare_graph.getNumOfComponents() == 0) + return; + auto run_for_all = [&](const auto func) { if (select_query->where()) From bf16b18f50f6b9baa038ddde3bb5200d4745cfd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:47:29 +0200 Subject: [PATCH 259/321] Update SubstituteColumnOptimizer.cpp --- src/Interpreters/SubstituteColumnOptimizer.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index ec51db56f14..925ded15857 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -13,10 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} namespace { From 410125bf8db6a672150c927cef960a181fa39a08 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:48:28 +0200 Subject: [PATCH 260/321] Fix MSan report in DatabaseReplicated --- src/Databases/DatabaseReplicated.cpp | 15 ++++++++++----- src/Databases/DatabaseReplicated.h | 2 +- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 21218e095bf..f127ccbc224 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -647,12 +647,13 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, { std::lock_guard lock{ddl_worker_mutex}; ddl_worker = std::make_unique(this, getContext()); + ddl_worker->startup(); + ddl_worker_initialized = true; } - ddl_worker->startup(); - ddl_worker_initialized = true; }); std::scoped_lock lock(mutex); - return startup_replicated_database_task = makeLoadTask(async_loader, {job}); + startup_replicated_database_task = makeLoadTask(async_loader, {job}); + return startup_replicated_database_task; } void DatabaseReplicated::waitDatabaseStarted() const @@ -1530,8 +1531,11 @@ void DatabaseReplicated::stopReplication() void DatabaseReplicated::shutdown() { stopReplication(); - ddl_worker_initialized = false; - ddl_worker = nullptr; + { + std::lock_guard lock{ddl_worker_mutex}; + ddl_worker_initialized = false; + ddl_worker = nullptr; + } DatabaseAtomic::shutdown(); } @@ -1679,6 +1683,7 @@ bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const /// It may update the metadata digest (both locally and in ZooKeeper) /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. /// We should not update metadata until the database is initialized. + std::lock_guard lock{ddl_worker_mutex}; return ddl_worker_initialized && ddl_worker->isCurrentlyActive(); } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 0f901538c33..27ab262d1f1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -155,7 +155,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; + mutable 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 10d003b1bf69575c32f55489a2659e7d71d9ad58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 03:54:00 +0200 Subject: [PATCH 261/321] Fix test `02310_clickhouse_local_INSERT_progress_profile_events` --- ...02310_clickhouse_local_INSERT_progress_profile_events.expect | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect index bde00b306cf..d5b2a278180 100755 --- a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect @@ -1,4 +1,6 @@ #!/usr/bin/expect -f +# Tags: no-debug, no-tsan, no-msan, no-asan, no-ubsan, no-s3-storage +# ^ it can be slower than 60 seconds # This is the regression for the concurrent access in ProgressIndication, # so it is important to read enough rows here (10e6). From 2e4cd203668671ac70d54b72f371e7cf361a9d76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 04:05:49 +0200 Subject: [PATCH 262/321] Remove test `02982_aggregation_states_destruction` --- .../02982_aggregation_states_destruction.reference | 1 - .../02982_aggregation_states_destruction.sh | 14 -------------- 2 files changed, 15 deletions(-) delete mode 100644 tests/queries/0_stateless/02982_aggregation_states_destruction.reference delete mode 100755 tests/queries/0_stateless/02982_aggregation_states_destruction.sh diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.reference b/tests/queries/0_stateless/02982_aggregation_states_destruction.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02982_aggregation_states_destruction.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.sh b/tests/queries/0_stateless/02982_aggregation_states_destruction.sh deleted file mode 100755 index 84183606d48..00000000000 --- a/tests/queries/0_stateless/02982_aggregation_states_destruction.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-random-settings - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -query_id="02982_$RANDOM" -$CLICKHOUSE_CLIENT --query_id $query_id --log_query_threads 1 --query="select number, uniq(number) from numbers_mt(1e7) group by number limit 100 format Null;" - -$CLICKHOUSE_CLIENT -q "system flush logs;" - -$CLICKHOUSE_CLIENT -q "select count() > 0 from system.query_thread_log where query_id = '$query_id' and current_database = currentDatabase() and thread_name = 'AggregDestruct';" From 51a01fd57660742f3287dea9d9416f0620143233 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 04:33:55 +0200 Subject: [PATCH 263/321] Fix error --- src/Parsers/ASTDataType.h | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h index d9427c2fd9e..71d3aeaa4eb 100644 --- a/src/Parsers/ASTDataType.h +++ b/src/Parsers/ASTDataType.h @@ -23,12 +23,14 @@ template std::shared_ptr makeASTDataType(const String & name, Args &&... args) { auto data_type = std::make_shared(); - data_type->name = name; - data_type->arguments = std::make_shared(); - data_type->children.push_back(data_type->arguments); - data_type->arguments->children = { std::forward(args)... }; + if constexpr (sizeof...(args)) + { + data_type->arguments = std::make_shared(); + data_type->children.push_back(data_type->arguments); + data_type->arguments->children = { std::forward(args)... }; + } return data_type; } From c2dae64df3946fd45fc4a0c863dd506329ffbb93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jul 2024 23:46:06 +0200 Subject: [PATCH 264/321] Challenge how optimistic was Maksim Kita --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 3 ++- tests/clickhouse-test | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86e04b2ab4e..c7a1a7e2739 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -186,7 +186,7 @@ class IColumn; M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \ M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \ - M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..0105e69a5e9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,7 +80,8 @@ static std::initializer_list Date: Thu, 25 Jul 2024 19:36:48 +0200 Subject: [PATCH 265/321] Update setting changes history --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0105e69a5e9..87eaeff0ca9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,7 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.8", {{"compile_expressions", false, true, "We believe that the LLVM infrastructure behind the JIT compiler is stable enough to enable this setting by default."}}}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, @@ -81,7 +82,6 @@ static std::initializer_list Date: Sat, 27 Jul 2024 04:41:46 +0200 Subject: [PATCH 266/321] Fix OOM in test runs --- .../02481_parquet_list_monotonically_increasing_offsets.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh index 55e6ac2f758..6fd6da69b70 100755 --- a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh +++ b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-ubsan, no-fasttest, no-tsan +# Tags: no-ubsan, no-fasttest, no-tsan, no-msan, no-asan # It is too slow under TSan +# It eats too much memory under ASan or MSan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From ab9270c31a728473ffcea68de05b869f2afbfd5b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 04:47:13 +0200 Subject: [PATCH 267/321] Update test --- ..._constraints_simple_optimization.reference | 8 +++--- .../01623_constraints_column_swap.reference | 26 +++++++++---------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index d267df2237f..84c872856ff 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -32,10 +32,10 @@ 1 1 0 -SELECT count() AS `count()` +SELECT count() FROM constraint_test_constants WHERE (b > 100) OR (c > 100) -SELECT count() AS `count()` +SELECT count() FROM constraint_test_constants WHERE c > 100 QUERY id: 0 @@ -53,7 +53,7 @@ QUERY id: 0 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 -SELECT count() AS `count()` +SELECT count() FROM constraint_test_constants WHERE c > 100 QUERY id: 0 @@ -71,7 +71,7 @@ QUERY id: 0 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 -SELECT count() AS `count()` +SELECT count() FROM constraint_test_constants QUERY id: 0 PROJECTION COLUMNS diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 555a4c93f70..d504a86365b 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -1,6 +1,6 @@ SELECT - (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, - (b AS b) + 3 AS `plus(b, 3)` + (b AS `cityHash64(a)`) + 10, + (b AS b) + 3 FROM column_swap_test_test WHERE b = 1 QUERY id: 0 @@ -59,8 +59,8 @@ QUERY id: 0 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT - (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, - (b AS b) + 3 AS `plus(b, 3)` + (b AS `cityHash64(a)`) + 10, + (b AS b) + 3 FROM column_swap_test_test WHERE b = 0 QUERY id: 0 @@ -89,8 +89,8 @@ QUERY id: 0 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT - (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, - (b AS b) + 3 AS `plus(b, 3)` + (b AS `cityHash64(a)`) + 10, + (b AS b) + 3 FROM column_swap_test_test WHERE b = 0 QUERY id: 0 @@ -119,8 +119,8 @@ QUERY id: 0 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT - (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, - (b AS b) + 3 AS `plus(b, 3)` + (b AS `cityHash64(a)`) + 10, + (b AS b) + 3 FROM column_swap_test_test WHERE b = 1 QUERY id: 0 @@ -148,7 +148,7 @@ QUERY id: 0 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 -SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` +SELECT (b AS `cityHash64(a)`) + 10 FROM column_swap_test_test WHERE b = 0 QUERY id: 0 @@ -171,8 +171,8 @@ QUERY id: 0 CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT - (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, - a AS a + (cityHash64(a) AS `cityHash64(a)`) + 10, + a FROM column_swap_test_test WHERE cityHash64(a) = 0 QUERY id: 0 @@ -203,8 +203,8 @@ QUERY id: 0 CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT - (cityHash64(a) AS b) + 10 AS `plus(b, 10)`, - a AS a + (cityHash64(a) AS b) + 10, + a FROM column_swap_test_test WHERE cityHash64(a) = 0 QUERY id: 0 From 46218b68ff47e079d5636be63f99a1deb5ad180b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 27 Jul 2024 06:32:26 +0200 Subject: [PATCH 268/321] Initialize the client_context after quota_key in clickhouse-client. --- programs/client/Client.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f2919db0308..3e613532f3a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1069,11 +1069,6 @@ void Client::processOptions(const OptionsDescription & options_description, global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::CLIENT); - /// In case of clickhouse-client the `client_context` can be just an alias for the `global_context`. - /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) - client_context = global_context; - initClientContext(); - global_context->setSettings(cmd_settings); /// Copy settings-related program options to config. @@ -1168,6 +1163,11 @@ void Client::processOptions(const OptionsDescription & options_description, if (options.count("opentelemetry-tracestate")) global_context->getClientTraceContext().tracestate = options["opentelemetry-tracestate"].as(); + + /// In case of clickhouse-client the `client_context` can be just an alias for the `global_context`. + /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) + client_context = global_context; + initClientContext(); } From 85249b47f18526d59791ffe65f1d247640f9854e Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 27 Jul 2024 11:56:20 +0200 Subject: [PATCH 269/321] Review changes --- .../functions/type-conversion-functions.md | 692 ++++++++++-------- 1 file changed, 405 insertions(+), 287 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 87d824ec5bb..ee253e46014 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -63,18 +63,19 @@ toInt8(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt8('0xc0fe');` +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), the result over or under flows. This is not considered an error. -For example: `SELECT toInt8(128) == -128;`, `SELECT toInt8(128.0) == -128;`, `SELECT toInt8('128') == -128;`. +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +For example: `SELECT toInt8(128) == -128;`. ::: **Returned value** @@ -93,15 +94,18 @@ Query: SELECT toInt8(-8), toInt8(-8.8), - toInt8('-8'); + toInt8('-8') +FORMAT vertical; ``` Result: ```response - ┌─toInt8(-8)─┬─toInt8(-8.8)─┬─toInt8('-8')─┐ -1. │ -8 │ -8 │ -8 │ - └────────────┴──────────────┴──────────────┘ +Row 1: +────── +toInt8(-8): -8 +toInt8(-8.8): -8 +toInt8('-8'): -8 ``` **See also** @@ -124,14 +128,17 @@ toInt8OrZero(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of ordinary Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -148,15 +155,17 @@ Query: ``` sql SELECT toInt8OrZero('-8'), - toInt8OrZero('abc'); + toInt8OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt8OrZero('-8')─┬─toInt8OrZero('abc')─┐ -1. │ -8 │ 0 │ - └────────────────────┴─────────────────────┘ +Row 1: +────── +toInt8OrZero('-8'): -8 +toInt8OrZero('abc'): 0 ``` **See also** @@ -167,7 +176,7 @@ Result: ## toInt8OrNull -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt8`](#toint8), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -179,14 +188,17 @@ toInt8OrNull(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -201,15 +213,19 @@ The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding Query: ``` sql -SELECT toInt8OrNull('-8'), toInt8OrNull('abc'); +SELECT + toInt8OrNull('-8'), + toInt8OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt8OrNull('-8')─┬─toInt8OrNull('abc')─┐ -1. │ -8 │ ᴺᵁᴸᴸ │ - └────────────────────┴─────────────────────┘ +Row 1: +────── +toInt8OrNull('-8'): -8 +toInt8OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -233,15 +249,19 @@ toInt8OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrDefault('0xc0fe', CAST('-1', 'Int8'));` -- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md) and the result over or under flows. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrDefault('0xc0fe', CAST('-1', 'Int8'));`. + +:::note +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -259,15 +279,17 @@ Query: ``` sql SELECT toInt8OrDefault('-8', CAST('-1', 'Int8')), - toInt8OrDefault('abc', CAST('-1', 'Int8')); + toInt8OrDefault('abc', CAST('-1', 'Int8')) +FORMAT vertical; ``` Result: ```response - ┌─toInt8OrDefault('-8', CAST('-1', 'Int8'))─┬─toInt8OrDefault('abc', CAST('-1', 'Int8'))─┐ -1. │ -8 │ -1 │ - └───────────────────────────────────────────┴────────────────────────────────────────────┘ +Row 1: +────── +toInt8OrDefault('-8', CAST('-1', 'Int8')): -8 +toInt8OrDefault('abc', CAST('-1', 'Int8')): -1 ``` **See also** @@ -290,18 +312,19 @@ toInt16(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');` +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. -For example: `SELECT toInt16(32768) == -32768;`, `SELECT toInt16(32768) == -32768;`, `SELECT toInt16('32768') == -32768;`. +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +For example: `SELECT toInt16(32768) == -32768;`. ::: **Returned value** @@ -320,15 +343,18 @@ Query: SELECT toInt16(-16), toInt16(-16.16), - toInt16('-16'); + toInt16('-16') +FORMAT vertical; ``` Result: ```response - ┌─toInt16(-16)─┬─toInt16(-16.16)─┬─toInt16('-16')─┐ -1. │ -16 │ -16 │ -16 │ - └──────────────┴─────────────────┴────────────────┘ +Row 1: +────── +toInt16(-16): -16 +toInt16(-16.16): -16 +toInt16('-16'): -16 ``` **See also** @@ -351,14 +377,17 @@ toInt16OrZero(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered as an error. +::: **Returned value** @@ -375,15 +404,17 @@ Query: ``` sql SELECT toInt16OrZero('-16'), - toInt16OrZero('abc'); + toInt16OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt16OrZero('-16')─┬─toInt16OrZero('abc')─┐ -1. │ -16 │ 0 │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt16OrZero('-16'): -16 +toInt16OrZero('abc'): 0 ``` **See also** @@ -394,7 +425,7 @@ Result: ## toInt16OrNull -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt16`](#toint16), this function converts an input value to a value of type [Int16](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -406,14 +437,17 @@ toInt16OrNull(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -430,15 +464,17 @@ Query: ``` sql SELECT toInt16OrNull('-16'), - toInt16OrNull('abc'); + toInt16OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt16OrNull('-16')─┬─toInt16OrNull('abc')─┐ -1. │ -16 │ ᴺᵁᴸᴸ │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt16OrNull('-16'): -16 +toInt16OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -449,7 +485,7 @@ Result: ## toInt16OrDefault -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toInt16`](#toint16), this function converts an input value to a value of type [Int16](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -462,15 +498,19 @@ toInt16OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrDefault('0xc0fe', CAST('-1', 'Int16'));` -- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrDefault('0xc0fe', CAST('-1', 'Int16'));`. + +:::note +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -486,15 +526,19 @@ Types for which the default value is returned: Query: ``` sql -SELECT toInt16OrDefault('-16', cast('-1' as Int16)), toInt16OrDefault('abc', cast('-1' as Int16)); +SELECT + toInt16OrDefault('-16', CAST('-1', 'Int16')), + toInt16OrDefault('abc', CAST('-1', 'Int16')) +FORMAT vertical; ``` Result: ```response - ┌─toInt16OrDefault('-16', CAST('-1', 'Int16'))─┬─toInt16OrDefault('abc', CAST('-1', 'Int16'))─┐ -1. │ -16 │ -1 │ - └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +Row 1: +────── +toInt16OrDefault('-16', CAST('-1', 'Int16')): -16 +toInt16OrDefault('abc', CAST('-1', 'Int16')): -1 ``` **See also** @@ -517,23 +561,19 @@ toInt32(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');` +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), the result over or under flows. This is not considered an error. -For example: -``` -SELECT toInt32(2147483648) == -2147483648; -SELECT toInt32(2147483648.0) == -2147483648; -SELECT toInt32('2147483648') == -2147483648; -``` +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +For example: `SELECT toInt32(2147483648) == -2147483648;` ::: **Returned value** @@ -553,14 +593,17 @@ SELECT toInt32(-32), toInt32(-32.32), toInt32('-32') +FORMAT vertical; ``` Result: ```response - ┌─toInt32(-32)─┬─toInt32(-32.32)─┬─toInt32('-32')─┐ -1. │ -32 │ -32 │ -32 │ - └──────────────┴─────────────────┴────────────────┘ +Row 1: +────── +toInt32(-32): -32 +toInt32(-32.32): -32 +toInt32('-32'): -32 ``` **See also** @@ -583,15 +626,17 @@ toInt32OrZero(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. +:::note +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -606,15 +651,19 @@ The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding Query: ``` sql -SELECT toInt32OrZero('-32'), toInt32OrZero('abc'); +SELECT + toInt32OrZero('-32'), + toInt32OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt32OrZero('-32')─┬─toInt32OrZero('abc')─┐ -1. │ -32 │ 0 │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt32OrZero('-32'): -32 +toInt32OrZero('abc'): 0 ``` **See also** @@ -624,7 +673,7 @@ Result: ## toInt32OrNull -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int32](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -636,14 +685,17 @@ toInt32OrNull(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -658,15 +710,19 @@ The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding Query: ``` sql -SELECT toInt32OrNull('-32'), toInt32OrNull('abc'); +SELECT + toInt32OrNull('-32'), + toInt32OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt32OrNull('-32')─┬─toInt32OrNull('abc')─┐ -1. │ -32 │ ᴺᵁᴸᴸ │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt32OrNull('-32'): -32 +toInt32OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -677,7 +733,7 @@ Result: ## toInt32OrDefault -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int32](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -690,15 +746,19 @@ toInt32OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrDefault('0xc0fe', CAST('-1', 'Int32'));` -- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrDefault('0xc0fe', CAST('-1', 'Int32'));`. + +:::note +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -714,15 +774,19 @@ Types for which the default value is returned: Query: ``` sql -SELECT toInt32OrDefault('-32', cast('-1' as Int32)), toInt32OrDefault('abc', cast('-1' as Int32)); +SELECT + toInt32OrDefault('-32', CAST('-1', 'Int32')), + toInt32OrDefault('abc', CAST('-1', 'Int32')) +FORMAT vertical; ``` Result: ```response - ┌─toInt32OrDefault('-32', CAST('-1', 'Int32'))─┬─toInt32OrDefault('abc', CAST('-1', 'Int32'))─┐ -1. │ -32 │ -1 │ - └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +Row 1: +────── +toInt32OrDefault('-32', CAST('-1', 'Int32')): -32 +toInt32OrDefault('abc', CAST('-1', 'Int32')): -1 ``` **See also** @@ -745,24 +809,19 @@ toInt64(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');` +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), the result over or under flows. This is not considered an error. -For example: - -``` -SELECT toInt64(9223372036854775808) == -9223372036854775808; -SELECT toInt64(9223372036854775808.0) == -9223372036854775808; -SELECT toInt64('9223372036854775808') == --9223372036854775808; -``` +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +For example: `SELECT toInt64(9223372036854775808) == -9223372036854775808;` ::: **Returned value** @@ -781,15 +840,18 @@ Query: SELECT toInt64(-64), toInt64(-64.64), - toInt64('-64'); + toInt64('-64') +FORMAT vertical; ``` Result: ```response - ┌─toInt64(-64)─┬─toInt64(-64.64)─┬─toInt64('-64')─┐ -1. │ -64 │ -64 │ -64 │ - └──────────────┴─────────────────┴────────────────┘ +Row 1: +────── +toInt64(-64): -64 +toInt64(-64.64): -64 +toInt64('-64'): -64 ``` **See also** @@ -812,14 +874,17 @@ toInt64OrZero(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -836,15 +901,17 @@ Query: ``` sql SELECT toInt64OrZero('-64'), - toInt64OrZero('abc'); + toInt64OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt64OrZero('-64')─┬─toInt64OrZero('abc')─┐ -1. │ -64 │ 0 │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt64OrZero('-64'): -64 +toInt64OrZero('abc'): 0 ``` **See also** @@ -855,7 +922,7 @@ Result: ## toInt64OrNull -Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int64](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -867,14 +934,17 @@ toInt64OrNull(x) - `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -891,15 +961,17 @@ Query: ``` sql SELECT toInt64OrNull('-64'), - toInt64OrNull('abc'); + toInt64OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt64OrNull('-64')─┬─toInt64OrNull('abc')─┐ -1. │ -64 │ ᴺᵁᴸᴸ │ - └──────────────────────┴──────────────────────┘ +Row 1: +────── +toInt64OrNull('-64'): -64 +toInt64OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -910,7 +982,7 @@ Result: ## toInt64OrDefault -Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns the default type value. +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int64](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -923,15 +995,19 @@ toInt64OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrDefault('0xc0fe', CAST('-1', 'Int64'));` -- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrDefault('0xc0fe', CAST('-1', 'Int64'));`. + +:::note +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -949,15 +1025,17 @@ Query: ``` sql SELECT toInt64OrDefault('-64', CAST('-1', 'Int64')), - toInt64OrDefault('abc', CAST('-1', 'Int64')); + toInt64OrDefault('abc', CAST('-1', 'Int64')) +FORMAT vertical; ``` Result: ```response - ┌─toInt64OrDefault('-64', CAST('-1', 'Int64'))─┬─toInt64OrDefault('abc', CAST('-1', 'Int64'))─┐ -1. │ -64 │ -1 │ - └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +Row 1: +────── +toInt64OrDefault('-64', CAST('-1', 'Int64')): -64 +toInt64OrDefault('abc', CAST('-1', 'Int64')): -1 ``` **See also** @@ -980,17 +1058,18 @@ toInt128(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt128('0xc0fe');` +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. ::: **Returned value** @@ -1009,15 +1088,18 @@ Query: SELECT toInt128(-128), toInt128(-128.8), - toInt128('-128'), + toInt128('-128') +FORMAT vertical; ``` Result: ```response - ┌─toInt128(-128)─┬─toInt128(-128.8)─┬─toInt128('-128')─┐ -1. │ -128 │ -128 │ -128 │ - └────────────────┴──────────────────┴──────────────────┘ +Row 1: +────── +toInt128(-128): -128 +toInt128(-128.8): -128 +toInt128('-128'): -128 ``` **See also** @@ -1040,14 +1122,17 @@ toInt128OrZero(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1064,15 +1149,17 @@ Query: ``` sql SELECT toInt128OrZero('-128'), - toInt128OrZero('abc'); + toInt128OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt128OrZero('-128')─┬─toInt128OrZero('abc')─┐ -1. │ -128 │ 0 │ - └────────────────────────┴───────────────────────┘ +Row 1: +────── +toInt128OrZero('-128'): -128 +toInt128OrZero('abc'): 0 ``` **See also** @@ -1083,7 +1170,7 @@ Result: ## toInt128OrNull -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int128](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -1095,14 +1182,17 @@ toInt128OrNull(x) - `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1119,15 +1209,17 @@ Query: ``` sql SELECT toInt128OrNull('-128'), - toInt128OrNull('abc'); + toInt128OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt128OrNull('-128')─┬─toInt128OrNull('abc')─┐ -1. │ -128 │ ᴺᵁᴸᴸ │ - └────────────────────────┴───────────────────────┘ +Row 1: +────── +toInt128OrNull('-128'): -128 +toInt128OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -1138,7 +1230,7 @@ Result: ## toInt128OrDefault -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int128](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -1151,15 +1243,19 @@ toInt128OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- (U)Int8/16/32/64/128/256. +- Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int128'));` -- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int128'));`. + +:::note +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1177,15 +1273,17 @@ Query: ``` sql SELECT toInt128OrDefault('-128', CAST('-1', 'Int128')), - toInt128OrDefault('abc', CAST('-1', 'Int128')); + toInt128OrDefault('abc', CAST('-1', 'Int128')) +FORMAT vertical; ``` Result: ```response - ┌─toInt128OrDefault('-128', CAST('-1', 'Int128'))─┬─toInt128OrDefault('abc', CAST('-1', 'Int128'))─┐ -1. │ -128 │ -1 │ - └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +Row 1: +────── +toInt128OrDefault('-128', CAST('-1', 'Int128')): -128 +toInt128OrDefault('abc', CAST('-1', 'Int128')): -1 ``` **See also** @@ -1208,17 +1306,18 @@ toInt256(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Unsupported types: -- Float values `NaN` and `Inf` throw an exception. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt256('0xc0fe');` +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256('0xc0fe');`. :::note -If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. ::: **Returned value** @@ -1237,15 +1336,18 @@ Query: SELECT toInt256(-256), toInt256(-256.256), - toInt256('-256'); + toInt256('-256') +FORMAT vertical; ``` Result: ```response - ┌─toInt256(-256)─┬─toInt256(-256.256)─┬─toInt256('-256')─┐ -1. │ -256 │ -256 │ -256 │ - └────────────────┴────────────────────┴──────────────────┘ +Row 1: +────── +toInt256(-256): -256 +toInt256(-256.256): -256 +toInt256('-256'): -256 ``` **See also** @@ -1268,14 +1370,17 @@ toInt256OrZero(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `0` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1292,15 +1397,17 @@ Query: ``` sql SELECT toInt256OrZero('-256'), - toInt256OrZero('abc'); + toInt256OrZero('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt256OrZero('-256')─┬─toInt256OrZero('abc')─┐ -1. │ -256 │ 0 │ - └────────────────────────┴───────────────────────┘ +Row 1: +────── +toInt256OrZero('-256'): -256 +toInt256OrZero('abc'): 0 ``` **See also** @@ -1311,7 +1418,7 @@ Result: ## toInt256OrNull -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int256](../data-types/int-uint.md) but returns `NULL` in case of an error. **Syntax** @@ -1323,14 +1430,17 @@ toInt256OrNull(x) - `x` — A String representation of a number. [String](../data-types/string.md). -Supported types: -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- String representations of (U)Int8/16/32/128/256. -Types for which `\N` is returned: -- String representations of ordinary Float32/64 values. -- String representations of Float values `NaN` and `Inf`. +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1347,15 +1457,17 @@ Query: ``` sql SELECT toInt256OrNull('-256'), - toInt256OrNull('abc'); + toInt256OrNull('abc') +FORMAT vertical; ``` Result: ```response - ┌─toInt256OrNull('-256')─┬─toInt256OrNull('abc')─┐ -1. │ -256 │ ᴺᵁᴸᴸ │ - └────────────────────────┴───────────────────────┘ +Row 1: +────── +toInt256OrNull('-256'): -256 +toInt256OrNull('abc'): ᴺᵁᴸᴸ ``` **See also** @@ -1366,7 +1478,7 @@ Result: ## toInt256OrDefault -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int256](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -1379,15 +1491,19 @@ toInt256OrDefault(expr, def) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). -Supported types: -- (U)Int8/16/32/64/128/256 -- Float* -- String representations of (U)Int8/16/32/128/256 +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. -Types for which the default value is returned: -- Float values `NaN` and `Inf` return the default value. +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf` - String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrDefault('0xc0fe', CAST('-1', 'Int256'));` -- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. + +:::note +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: **Returned value** @@ -1405,15 +1521,17 @@ Query: ``` sql SELECT toInt256OrDefault('-256', CAST('-1', 'Int256')), - toInt256OrDefault('abc', CAST('-1', 'Int256')); + toInt256OrDefault('abc', CAST('-1', 'Int256')) +FORMAT vertical; ``` Result: ```response - ┌─toInt256OrDefault('-256', CAST('-1', 'Int256'))─┬─toInt256OrDefault('abc', CAST('-1', 'Int256'))─┐ -1. │ -256 │ -1 │ - └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +Row 1: +────── +toInt256OrDefault('-256', CAST('-1', 'Int256')): -256 +toInt256OrDefault('abc', CAST('-1', 'Int256')): -1 ``` **See also** From 5f0c40fafcefc8eba63ca1a872a6aa49939dcdaa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 12:10:41 +0200 Subject: [PATCH 270/321] Apply review comments --- src/Common/ZooKeeper/ZooKeeper.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 7448d73cbbc..064ac2261ec 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -154,14 +154,14 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr ShuffleHosts node{optimal_host}; std::unique_ptr new_impl = std::make_unique(node, args, zk_log); - if (auto new_node_idx = new_impl->getConnectedNodeIdx(); new_node_idx) + auto new_node_idx = new_impl->getConnectedNodeIdx(); + chassert(new_node_idx.has_value()); + + /// Maybe the node was unavailable when getting AZs first time, update just in case + if (args.availability_zone_autodetect && availability_zones[*new_node_idx].empty()) { - /// Maybe the node was unavailable when getting AZs first time, update just in case - if (args.availability_zone_autodetect && availability_zones[*new_node_idx].empty()) - { - availability_zones[*new_node_idx] = new_impl->tryGetAvailabilityZone(); - LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[*new_node_idx]); - } + availability_zones[*new_node_idx] = new_impl->tryGetAvailabilityZone(); + LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[*new_node_idx]); } optimal_impl = std::move(new_impl); From 28b83a38f2671c2b85cfb2b116371ec754f39e75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 13:08:48 +0200 Subject: [PATCH 271/321] Remove harmful stuff from tests --- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 -- tests/queries/0_stateless/02930_client_file_log_comment.sh | 2 -- tests/queries/0_stateless/02941_variant_type_1.sh | 2 -- tests/queries/0_stateless/02941_variant_type_2.sh | 2 -- tests/queries/0_stateless/02941_variant_type_3.sh | 2 -- tests/queries/0_stateless/02941_variant_type_4.sh | 2 -- tests/queries/0_stateless/02941_variant_type_alters.sh | 2 -- tests/queries/0_stateless/02943_variant_read_subcolumns.sh | 3 --- ...943_variant_type_with_different_local_and_global_order.sh | 2 -- tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh | 2 -- tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh | 2 -- tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh | 2 -- .../0_stateless/03039_dynamic_aggregating_merge_tree.sh | 2 -- .../0_stateless/03039_dynamic_collapsing_merge_tree.sh | 2 -- .../0_stateless/03039_dynamic_replacing_merge_tree.sh | 2 -- .../queries/0_stateless/03039_dynamic_summing_merge_tree.sh | 2 -- .../03039_dynamic_versioned_collapsing_merge_tree.sh | 2 -- tests/queries/0_stateless/03041_dynamic_type_check_table.sh | 3 --- .../queries/0_stateless/03201_variant_null_map_subcolumn.sh | 5 +---- .../queries/0_stateless/03202_dynamic_null_map_subcolumn.sh | 2 -- 20 files changed, 1 insertion(+), 44 deletions(-) diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index dacb609d790..804f51ea8a2 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -1,8 +1,6 @@ #!/usr/bin/env bash # Tags: no-random-settings -unset CLICKHOUSE_LOG_COMMENT - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.sh b/tests/queries/0_stateless/02930_client_file_log_comment.sh index 3d0df6de9b4..1221c8a6521 100755 --- a/tests/queries/0_stateless/02930_client_file_log_comment.sh +++ b/tests/queries/0_stateless/02930_client_file_log_comment.sh @@ -1,8 +1,6 @@ #!/usr/bin/env bash CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 723de45eaad..c12bece6d54 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 8453bce98dc..e93dfac8510 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index 990eb25b5be..cc0fde5b689 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b8f619694b0..93a1770d05e 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/02941_variant_type_alters.sh b/tests/queries/0_stateless/02941_variant_type_alters.sh index 7c151d1fe9e..0b3ab8e925b 100755 --- a/tests/queries/0_stateless/02941_variant_type_alters.sh +++ b/tests/queries/0_stateless/02941_variant_type_alters.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns.sh index 5ca8dd5f36f..21ab877eaf3 100755 --- a/tests/queries/0_stateless/02943_variant_read_subcolumns.sh +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns.sh @@ -2,8 +2,6 @@ # 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 @@ -34,4 +32,3 @@ echo "MergeTree wide" $CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) 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/02943_variant_type_with_different_local_and_global_order.sh b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh index 1d88757a5d6..4aa4b11cecb 100755 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh @@ -2,8 +2,6 @@ # Tags: long, no-debug CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh index aabba731816..ed548ae74e9 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh index 872f4c20a98..95dafcf5832 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh index 96276c96add..a3c2d93e568 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03039_dynamic_aggregating_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_aggregating_merge_tree.sh index b8760ec0e1d..9ea86105a3a 100755 --- a/tests/queries/0_stateless/03039_dynamic_aggregating_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_aggregating_merge_tree.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03039_dynamic_collapsing_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_collapsing_merge_tree.sh index 881c9ec64cc..9a2a6dd957c 100755 --- a/tests/queries/0_stateless/03039_dynamic_collapsing_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_collapsing_merge_tree.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03039_dynamic_replacing_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_replacing_merge_tree.sh index fc9039ac98c..0199035a3df 100755 --- a/tests/queries/0_stateless/03039_dynamic_replacing_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_replacing_merge_tree.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03039_dynamic_summing_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_summing_merge_tree.sh index f9da70e95ca..e2ea5bc3466 100755 --- a/tests/queries/0_stateless/03039_dynamic_summing_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_summing_merge_tree.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh index 60248f4453a..43607cf95a8 100755 --- a/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh @@ -2,8 +2,6 @@ # 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 diff --git a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh index 3d802485be3..c8bd533e253 100755 --- a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh +++ b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh @@ -2,8 +2,6 @@ # 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 @@ -42,4 +40,3 @@ 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/03201_variant_null_map_subcolumn.sh b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh index 8231691e184..050be8359ad 100755 --- a/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh @@ -2,8 +2,6 @@ # 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 @@ -17,7 +15,7 @@ function test() $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id" $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id" $CH_CLIENT -q "select id from test where v.UInt64 is null order by id" - + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" $CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" @@ -41,4 +39,3 @@ echo "MergeTree wide" $CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) 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/03202_dynamic_null_map_subcolumn.sh b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh index 968c9e5271f..bbe25ea72bb 100755 --- a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh @@ -2,8 +2,6 @@ # 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 From 8f96858df1be98d4707a2cf5f821c77428134e4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 13:11:53 +0200 Subject: [PATCH 272/321] Fix test --- tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh index 8231691e184..57dc36d8a8f 100755 --- a/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh @@ -17,8 +17,7 @@ function test() $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id" $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id" $CH_CLIENT -q "select id from test where v.UInt64 is null order by id" - - $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(250000) settings min_insert_block_size_rows=100000, min_insert_block_size_bytes=0" $CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id format Null" @@ -41,4 +40,3 @@ echo "MergeTree wide" $CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) 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 554cf91f4bb2d8d2272428c3f2cfbb4c3556d4b1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 27 Jul 2024 14:42:23 +0200 Subject: [PATCH 273/321] Add missing call applyCmdOptions(). --- programs/local/LocalServer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e60c8ef6085..ce0e179939d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -735,11 +735,13 @@ void LocalServer::processConfig() /// there is separate context for Buffer tables). adjustSettings(); applySettingsOverridesForLocal(global_context); + applyCmdOptions(global_context); /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(getClientConfiguration()); - applyCmdOptions(global_context); + /// Command-line parameters can override settings from the default profile. + applyCmdSettings(global_context); /// We load temporary database first, because projections need it. DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); From 6bbd9e6d67800fcfcb7105b386a78ba8c09b709a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 15:43:09 +0200 Subject: [PATCH 274/321] Update 02930_client_file_log_comment.sh --- tests/queries/0_stateless/02930_client_file_log_comment.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.sh b/tests/queries/0_stateless/02930_client_file_log_comment.sh index 1221c8a6521..50cd587e4b5 100755 --- a/tests/queries/0_stateless/02930_client_file_log_comment.sh +++ b/tests/queries/0_stateless/02930_client_file_log_comment.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment, because the test has to set its own +CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh From b0c2991b84dfeadaae099f102f8ef4d9e58844d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 15:43:38 +0200 Subject: [PATCH 275/321] Update 01187_set_profile_as_setting.sh --- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index 804f51ea8a2..42f596c45d6 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -2,6 +2,8 @@ # Tags: no-random-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment, because the test has to use the readonly mode +CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 3c4902d9d327bdf2e9d22606f163cd137edd4b49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 16:04:08 +0200 Subject: [PATCH 276/321] Split `01651_lc_insert_tiny_log` --- .../01651_lc_insert_tiny_log.reference | 12 ----- .../0_stateless/01651_lc_insert_tiny_log.sql | 48 ------------------- .../01651_lc_insert_tiny_log_1.reference | 4 ++ .../01651_lc_insert_tiny_log_1.sql | 16 +++++++ .../01651_lc_insert_tiny_log_2.reference | 4 ++ .../01651_lc_insert_tiny_log_2.sql | 16 +++++++ .../01651_lc_insert_tiny_log_3.reference | 4 ++ .../01651_lc_insert_tiny_log_3.sql | 16 +++++++ 8 files changed, 60 insertions(+), 60 deletions(-) delete mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log.reference delete mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log.sql create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_1.reference create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_1.sql create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_2.reference create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_2.sql create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_3.reference create mode 100644 tests/queries/0_stateless/01651_lc_insert_tiny_log_3.sql diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference b/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference deleted file mode 100644 index 5cc8909b6c8..00000000000 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.reference +++ /dev/null @@ -1,12 +0,0 @@ -10000000 -10000000 1274991808 -30000000 -30000000 3824991808 -10000000 -10000000 1274991808 -30000000 -30000000 3824991808 -10000000 -10000000 1274991808 -30000000 -30000000 3824991808 diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql deleted file mode 100644 index d11c9120c61..00000000000 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql +++ /dev/null @@ -1,48 +0,0 @@ -set allow_suspicious_low_cardinality_types=1; -drop table if exists perf_lc_num; - -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = TinyLog; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -drop table if exists perf_lc_num; - - -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = Log; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -drop table if exists perf_lc_num; - - -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = StripeLog; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); - -select sum(length(arr)) from perf_lc_num; -select sum(length(arr)), sum(num) from perf_lc_num; - -drop table if exists perf_lc_num; - - diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.reference b/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.reference new file mode 100644 index 00000000000..0715a4212ed --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.reference @@ -0,0 +1,4 @@ +10000000 +10000000 1274991808 +30000000 +30000000 3824991808 diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.sql new file mode 100644 index 00000000000..142d3b21b47 --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_1.sql @@ -0,0 +1,16 @@ +set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '31M'; +drop table if exists perf_lc_num; + +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = TinyLog; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +drop table if exists perf_lc_num; diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.reference b/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.reference new file mode 100644 index 00000000000..0715a4212ed --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.reference @@ -0,0 +1,4 @@ +10000000 +10000000 1274991808 +30000000 +30000000 3824991808 diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.sql new file mode 100644 index 00000000000..e6b68e1d682 --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_2.sql @@ -0,0 +1,16 @@ +set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '31M'; +drop table if exists perf_lc_num; + +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = Log; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +drop table if exists perf_lc_num; diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.reference b/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.reference new file mode 100644 index 00000000000..0715a4212ed --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.reference @@ -0,0 +1,4 @@ +10000000 +10000000 1274991808 +30000000 +30000000 3824991808 diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.sql new file mode 100644 index 00000000000..85a42163897 --- /dev/null +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log_3.sql @@ -0,0 +1,16 @@ +set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '31M'; +drop table if exists perf_lc_num; + +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = StripeLog; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000); + +select sum(length(arr)) from perf_lc_num; +select sum(length(arr)), sum(num) from perf_lc_num; + +drop table if exists perf_lc_num; From 0de2b07119b6ab61a25df5f7b641282dda5e63d7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 16:19:20 +0200 Subject: [PATCH 277/321] Debug test `02490_benchmark_max_consecutive_errors` --- .../02490_benchmark_max_consecutive_errors.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh index 3c046020773..f747b3156a5 100755 --- a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh +++ b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh @@ -5,4 +5,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh $CLICKHOUSE_BENCHMARK -q 'select throwIf(1)' |& grep '^An error occurred while processing the query.*Exception:' -c -$CLICKHOUSE_BENCHMARK --max-consecutive-errors 10 -q 'select throwIf(1)' |& grep '^An error occurred while processing the query.*Exception:' -c +RES=$($CLICKHOUSE_BENCHMARK --max-consecutive-errors 10 -q 'select throwIf(1)' |& tee "${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.log" | grep '^An error occurred while processing the query.*Exception:' -c) + +if [ "$RES" -eq 10 ] +then + echo "$RES" +else + cat "${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.log" +fi From 23cd9f2224cea2a52572a6ad4f1ba3f879dacd48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 17:17:54 +0200 Subject: [PATCH 278/321] Fix bad test @Demilivor --- programs/server/config.d/session_log.xml | 1 + ...ce => 02833_concurrent_sessions.reference} | 32 ++++++++--------- ...ssions.sh => 02833_concurrent_sessions.sh} | 34 +++++++++---------- 3 files changed, 34 insertions(+), 33 deletions(-) create mode 120000 programs/server/config.d/session_log.xml rename tests/queries/0_stateless/{02833_concurrrent_sessions.reference => 02833_concurrent_sessions.reference} (84%) rename tests/queries/0_stateless/{02833_concurrrent_sessions.sh => 02833_concurrent_sessions.sh} (88%) diff --git a/programs/server/config.d/session_log.xml b/programs/server/config.d/session_log.xml new file mode 120000 index 00000000000..7678cc33129 --- /dev/null +++ b/programs/server/config.d/session_log.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/session_log.xml \ No newline at end of file diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrent_sessions.reference similarity index 84% rename from tests/queries/0_stateless/02833_concurrrent_sessions.reference rename to tests/queries/0_stateless/02833_concurrent_sessions.reference index bfe507e8eac..904e8cefc6e 100644 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ b/tests/queries/0_stateless/02833_concurrent_sessions.reference @@ -1,34 +1,34 @@ sessions: -150 +45 port_0_sessions: 0 address_0_sessions: 0 tcp_sessions -60 +18 http_sessions -30 +9 http_with_session_id_sessions -30 -my_sql_sessions -30 +9 +mysql_sessions +9 Corresponding LoginSuccess/Logout -10 +3 LoginFailure -10 +3 Corresponding LoginSuccess/Logout -10 +3 LoginFailure -10 +3 Corresponding LoginSuccess/Logout -10 +3 LoginFailure -10 +3 Corresponding LoginSuccess/Logout -10 +3 LoginFailure -10 +3 Corresponding LoginSuccess/Logout -10 +3 LoginFailure -10 +3 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrent_sessions.sh similarity index 88% rename from tests/queries/0_stateless/02833_concurrrent_sessions.sh rename to tests/queries/0_stateless/02833_concurrent_sessions.sh index c5b6204529b..846661cfeed 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrent_sessions.sh @@ -37,11 +37,11 @@ done # These functions try to create a session with successful login and logout. # Sleep a small, random amount of time to make concurrency more intense. # and try to login with an invalid password. -function tcp_session() +function tcp_session() { local user=$1 local i=0 - while (( (i++) < 10 )); do + while (( (i++) < 3 )); do # login logout ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" # login failure @@ -49,11 +49,11 @@ function tcp_session() done } -function http_session() +function http_session() { local user=$1 local i=0 - while (( (i++) < 10 )); do + while (( (i++) < 3 )); do # login logout ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" @@ -62,11 +62,11 @@ function http_session() done } -function http_with_session_id_session() +function http_with_session_id_session() { local user=$1 local i=0 - while (( (i++) < 10 )); do + while (( (i++) < 3 )); do # login logout ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" @@ -75,11 +75,11 @@ function http_with_session_id_session() done } -function mysql_session() +function mysql_session() { local user=$1 local i=0 - while (( (i++) < 10 )); do + while (( (i++) < 3 )); do # login logout ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" @@ -97,29 +97,29 @@ export -f http_with_session_id_session; export -f mysql_session; for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & + tcp_session ${user} >/dev/null 2>&1 & done for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & + http_session ${user} >/dev/null 2>&1 & done for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & + http_with_session_id_session ${user} >/dev/null 2>&1 & done for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & + mysql_session ${user} >/dev/null 2>&1 & done wait ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -echo "sessions:" +echo "sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" -echo "port_0_sessions:" +echo "port_0_sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" echo "address_0_sessions:" @@ -131,13 +131,13 @@ echo "http_sessions" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" echo "http_with_session_id_sessions" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" +echo "mysql_sessions" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" for user in "${ALL_USERS[@]}"; do ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" + echo "Corresponding LoginSuccess/Logout" ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" done From f2a43b62b0af1aa1b41aae2cb23ad20b65fdd2e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 17:39:39 +0200 Subject: [PATCH 279/321] Add a separate test for exception handling --- .../0_stateless/03212_thousand_exceptions.reference | 1 + tests/queries/0_stateless/03212_thousand_exceptions.sh | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03212_thousand_exceptions.reference create mode 100755 tests/queries/0_stateless/03212_thousand_exceptions.sh diff --git a/tests/queries/0_stateless/03212_thousand_exceptions.reference b/tests/queries/0_stateless/03212_thousand_exceptions.reference new file mode 100644 index 00000000000..2409ba16fcb --- /dev/null +++ b/tests/queries/0_stateless/03212_thousand_exceptions.reference @@ -0,0 +1 @@ +('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.')('.') \ No newline at end of file diff --git a/tests/queries/0_stateless/03212_thousand_exceptions.sh b/tests/queries/0_stateless/03212_thousand_exceptions.sh new file mode 100755 index 00000000000..0a6abf35c10 --- /dev/null +++ b/tests/queries/0_stateless/03212_thousand_exceptions.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# This should not be too slow, even under sanitizers. +yes "SELECT throwIf(1); SELECT '.' FORMAT Values;" | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery --ignore-error 2>/dev/null From 7ab1664fe543e64680ac6bea866d9f2c8052ba1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 17:46:31 +0200 Subject: [PATCH 280/321] Debug test `01600_parts_states_metrics_long` --- .../01600_parts_states_metrics_long.sh | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 2e47034e528..cee7de3b1db 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -15,13 +15,22 @@ verify_sql="SELECT # In case of test failure, this code will do infinite loop and timeout. verify() { - while true + for i in {1..100} do result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) - [ "$result" = "1" ] && break + [ "$result" = "1" ] && echo "$result" && break sleep 0.1 + + if [[ $i -eq 100 ]] + then + $CLICKHOUSE_CLIENT --query " + SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; + SELECT sum(active), sum(NOT active) FROM system.parts; + SELECT sum(active), sum(NOT active) FROM system.projection_parts; + SELECT count() FROM system.dropped_tables_parts; + " + fi done - echo 1 } $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS test_table" From 4ec4b1f2f3f1806d479825e1e17d335b811555b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 17:57:09 +0200 Subject: [PATCH 281/321] Faster test `02231_buffer_aggregate_states_leak` --- .../0_stateless/02231_buffer_aggregate_states_leak.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql index 3d6b69fe161..85915e3790e 100644 --- a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql +++ b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan drop table if exists buffer_02231; drop table if exists out_02231; @@ -11,8 +11,8 @@ create table buffer_02231 ( key Int, v1 AggregateFunction(groupArray, String) -) engine=Buffer(currentDatabase(), 'out_02231', - /* layers= */1, +) engine = Buffer(currentDatabase(), 'out_02231', + /* layers= */ 1, /* min/max time */ 86400, 86400, /* min/max rows */ 1e9, 1e9, /* min/max bytes */ 1e12, 1e12, @@ -29,7 +29,7 @@ from in_02231 group by key; set optimize_trivial_insert_select = 1; -insert into in_02231 select * from numbers(10e6) settings max_memory_usage='400Mi', max_threads=1; +insert into in_02231 select * from numbers(5e6) settings max_memory_usage='30Mi', max_threads=1, min_insert_block_size_rows = 10000, min_insert_block_size_bytes = '10Mi'; drop table buffer_02231; drop table out_02231; From 5917210a0247923e4745a6b5b063790fd41202b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 19:51:05 +0200 Subject: [PATCH 282/321] Update 01600_parts_states_metrics_long.sh --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index cee7de3b1db..f535d888446 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -17,13 +17,13 @@ verify() { for i in {1..100} do - result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) + result=$( $CLICKHOUSE_CLIENT --query="$verify_sql" ) [ "$result" = "1" ] && echo "$result" && break sleep 0.1 if [[ $i -eq 100 ]] then - $CLICKHOUSE_CLIENT --query " + $CLICKHOUSE_CLIENT --multiquery " SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; SELECT sum(active), sum(NOT active) FROM system.parts; SELECT sum(active), sum(NOT active) FROM system.projection_parts; From 5ac26d2115c9e3e777990fc932bf03e83a2effee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 20:36:34 +0200 Subject: [PATCH 283/321] Fix curiosities in TimerDescriptor --- src/Common/TimerDescriptor.cpp | 44 +++++++++++++++++++++++++--------- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 248febe226e..2890ddec14c 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -1,11 +1,12 @@ #if defined(OS_LINUX) + #include #include #include -#include #include + namespace DB { @@ -19,15 +20,13 @@ namespace ErrorCodes TimerDescriptor::TimerDescriptor(int clockid, int flags) { - timer_fd = timerfd_create(clockid, flags); + timer_fd = timerfd_create(clockid, flags | TFD_NONBLOCK); if (timer_fd == -1) - throw Exception(ErrorCodes::CANNOT_CREATE_TIMER, "Cannot create timer_fd descriptor"); - - if (-1 == fcntl(timer_fd, F_SETFL, O_NONBLOCK)) - throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set O_NONBLOCK for timer_fd"); + throw ErrnoException(ErrorCodes::CANNOT_CREATE_TIMER, "Cannot create timer_fd descriptor"); } -TimerDescriptor::TimerDescriptor(TimerDescriptor && other) noexcept : timer_fd(other.timer_fd) +TimerDescriptor::TimerDescriptor(TimerDescriptor && other) noexcept + : timer_fd(other.timer_fd) { other.timer_fd = -1; } @@ -43,13 +42,16 @@ TimerDescriptor::~TimerDescriptor() /// Do not check for result cause cannot throw exception. if (timer_fd != -1) { - int err = close(timer_fd); - chassert(!err || errno == EINTR); + if (0 != ::close(timer_fd)) + std::terminate(); } } void TimerDescriptor::reset() const { + if (timer_fd == -1) + return; + itimerspec spec; spec.it_interval.tv_nsec = 0; spec.it_interval.tv_sec = 0; @@ -66,25 +68,44 @@ void TimerDescriptor::reset() const void TimerDescriptor::drain() const { + if (timer_fd == -1) + return; + /// It is expected that socket returns 8 bytes when readable. /// Read in loop anyway cause signal may interrupt read call. + + /// man timerfd_create: + /// If the timer has already expired one or more times since its settings were last modified using timerfd_settime(), + /// or since the last successful read(2), then the buffer given to read(2) returns an unsigned 8-byte integer (uint64_t) + /// containing the number of expirations that have occurred. + /// (The returned value is in host byte order—that is, the native byte order for integers on the host machine.) uint64_t buf; while (true) { ssize_t res = ::read(timer_fd, &buf, sizeof(buf)); if (res < 0) { + /// man timerfd_create: + /// If no timer expirations have occurred at the time of the read(2), + /// then the call either blocks until the next timer expiration, or fails with the error EAGAIN + /// if the file descriptor has been made nonblocking + /// (via the use of the fcntl(2) F_SETFL operation to set the O_NONBLOCK flag). if (errno == EAGAIN) break; - if (errno != EINTR) - throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd"); + /// A signal happened, need to retry. + if (errno == EINTR) + continue; + + throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd"); } } } void TimerDescriptor::setRelative(uint64_t usec) const { + chassert(timer_fd >= 0); + static constexpr uint32_t TIMER_PRECISION = 1e6; itimerspec spec; @@ -103,4 +124,5 @@ void TimerDescriptor::setRelative(Poco::Timespan timespan) const } } + #endif From 299d0fb39eaf3698994a98a65b7f01f191d2b65d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 20:43:15 +0200 Subject: [PATCH 284/321] Fix curiosities in TimerDescriptor --- src/Common/TimerDescriptor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 2890ddec14c..cb34170191a 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -39,7 +39,6 @@ TimerDescriptor & TimerDescriptor::operator=(DB::TimerDescriptor && other) noexc TimerDescriptor::~TimerDescriptor() { - /// Do not check for result cause cannot throw exception. if (timer_fd != -1) { if (0 != ::close(timer_fd)) From 377b78119462e3188f6261ae1c3203c999f75159 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 20:47:56 +0200 Subject: [PATCH 285/321] Fix curiosities in TimerDescriptor --- src/Common/TimerDescriptor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index cb34170191a..ba9f293d147 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -18,9 +18,9 @@ namespace ErrorCodes extern const int CANNOT_READ_FROM_SOCKET; } -TimerDescriptor::TimerDescriptor(int clockid, int flags) +TimerDescriptor::TimerDescriptor() { - timer_fd = timerfd_create(clockid, flags | TFD_NONBLOCK); + timer_fd = timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK | TFD_CLOEXEC); if (timer_fd == -1) throw ErrnoException(ErrorCodes::CANNOT_CREATE_TIMER, "Cannot create timer_fd descriptor"); } From 00b0fa961a225e8d95561d6e0114cac1d86a9a6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 20:50:21 +0200 Subject: [PATCH 286/321] Removed useless parameters --- src/Common/TimerDescriptor.h | 2 +- src/QueryPipeline/RemoteQueryExecutor.h | 1 - src/QueryPipeline/RemoteQueryExecutorReadContext.h | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index 0292f85d770..8d3b22868ac 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -12,7 +12,7 @@ private: int timer_fd; public: - explicit TimerDescriptor(int clockid = CLOCK_MONOTONIC, int flags = 0); + TimerDescriptor(); ~TimerDescriptor(); TimerDescriptor(const TimerDescriptor &) = delete; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 04a59cc3b7e..78e0b02c38b 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -8,7 +8,6 @@ #include #include #include -#include #include diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index b8aa8bb9111..fbd55278f86 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -69,7 +69,7 @@ private: /// * timer is a timerfd descriptor to manually check socket timeout /// * pipe_fd is a pipe we use to cancel query and socket polling by executor. /// We put those descriptors into our own epoll which is used by external executor. - TimerDescriptor timer{CLOCK_MONOTONIC, 0}; + TimerDescriptor timer; Poco::Timespan timeout; AsyncEventTimeoutType timeout_type; std::atomic_bool is_timer_alarmed = false; From 0967bfadd13c358764cf33d3cd78456a129faf91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 20:58:53 +0200 Subject: [PATCH 287/321] Simpler --- src/Common/TimerDescriptor.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index ba9f293d147..dd6ae70511e 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -51,11 +51,7 @@ void TimerDescriptor::reset() const if (timer_fd == -1) return; - itimerspec spec; - spec.it_interval.tv_nsec = 0; - spec.it_interval.tv_sec = 0; - spec.it_value.tv_sec = 0; - spec.it_value.tv_nsec = 0; + itimerspec spec{}; if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr)) throw ErrnoException(ErrorCodes::CANNOT_SET_TIMER_PERIOD, "Cannot reset timer_fd"); From 3dfa32498ce33565deb3d2009c0a5053c46cce98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 21:06:08 +0200 Subject: [PATCH 288/321] Add assert --- src/Common/TimerDescriptor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index dd6ae70511e..c7598546799 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -94,6 +94,8 @@ void TimerDescriptor::drain() const throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd"); } + + chassert(res == sizeof(buf)); } } From e0291e3a06d1f5ccd51f894d2cc9c3582911dc94 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 21:16:11 +0200 Subject: [PATCH 289/321] Prevent closing a file twice, even in case of errors --- src/IO/AsynchronousReadBufferFromFile.cpp | 3 +++ src/IO/MMapReadBufferFromFile.cpp | 3 +++ src/IO/MMappedFile.cpp | 3 +++ src/IO/OpenedFile.cpp | 4 +++- src/IO/ReadBufferFromFile.cpp | 3 +++ src/IO/WriteBufferFromFile.cpp | 3 +++ 6 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/IO/AsynchronousReadBufferFromFile.cpp b/src/IO/AsynchronousReadBufferFromFile.cpp index c6fe16a7f14..4d148a5c9f9 100644 --- a/src/IO/AsynchronousReadBufferFromFile.cpp +++ b/src/IO/AsynchronousReadBufferFromFile.cpp @@ -93,7 +93,10 @@ void AsynchronousReadBufferFromFile::close() return; if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; } diff --git a/src/IO/MMapReadBufferFromFile.cpp b/src/IO/MMapReadBufferFromFile.cpp index d3eb11c920d..d0865269f1b 100644 --- a/src/IO/MMapReadBufferFromFile.cpp +++ b/src/IO/MMapReadBufferFromFile.cpp @@ -77,7 +77,10 @@ void MMapReadBufferFromFile::close() finish(); if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; metric_increment.destroy(); diff --git a/src/IO/MMappedFile.cpp b/src/IO/MMappedFile.cpp index 7249a25decb..26949061274 100644 --- a/src/IO/MMappedFile.cpp +++ b/src/IO/MMappedFile.cpp @@ -69,7 +69,10 @@ void MMappedFile::close() finish(); if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; metric_increment.destroy(); diff --git a/src/IO/OpenedFile.cpp b/src/IO/OpenedFile.cpp index 4677a8259db..c51dc430b35 100644 --- a/src/IO/OpenedFile.cpp +++ b/src/IO/OpenedFile.cpp @@ -67,11 +67,13 @@ void OpenedFile::close() return; if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; metric_increment.destroy(); } } - diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index cb987171bad..2e5b9fcf753 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -88,7 +88,10 @@ void ReadBufferFromFile::close() return; if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; metric_increment.destroy(); diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 37b1161356f..f1825ce1e22 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -116,7 +116,10 @@ void WriteBufferFromFile::close() finalize(); if (0 != ::close(fd)) + { + fd = -1; throw Exception(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file"); + } fd = -1; metric_increment.destroy(); From 408daa0bf44dde0213f9c5bfd722a411e68e6dd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 23:00:32 +0200 Subject: [PATCH 290/321] Update TimerDescriptor.cpp --- src/Common/TimerDescriptor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index c7598546799..b36ea4059cb 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -14,7 +14,6 @@ namespace ErrorCodes { extern const int CANNOT_CREATE_TIMER; extern const int CANNOT_SET_TIMER_PERIOD; - extern const int CANNOT_FCNTL; extern const int CANNOT_READ_FROM_SOCKET; } From 61cc5a1e5440c126098546baee4a29af1a3df9ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 23:17:38 +0200 Subject: [PATCH 291/321] Add retries to test `02911_backup_restore_keeper_map` --- programs/server/config.d/backups.xml | 13 ++++++++++++ .../server/config.d/enable_keeper_map.xml | 1 + .../02911_backup_restore_keeper_map.sh | 20 +++++++++++++++---- 3 files changed, 30 insertions(+), 4 deletions(-) create mode 100644 programs/server/config.d/backups.xml create mode 120000 programs/server/config.d/enable_keeper_map.xml diff --git a/programs/server/config.d/backups.xml b/programs/server/config.d/backups.xml new file mode 100644 index 00000000000..382794e1123 --- /dev/null +++ b/programs/server/config.d/backups.xml @@ -0,0 +1,13 @@ + + + + + local + /tmp/backups/ + + + + + backups + + diff --git a/programs/server/config.d/enable_keeper_map.xml b/programs/server/config.d/enable_keeper_map.xml new file mode 120000 index 00000000000..47c36159882 --- /dev/null +++ b/programs/server/config.d/enable_keeper_map.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/enable_keeper_map.xml \ No newline at end of file diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index ccdf52a6e23..385583acbbe 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -11,11 +11,23 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE $database_name.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; CREATE TABLE $database_name.02911_backup_restore_keeper_map2 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; -- table using same Keeper path as 02911_backup_restore_keeper_map1 CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; - - INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; - INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; " +# KeeperMap table engine doesn't have internal retries for interaction with Keeper. Do it on our own, otherwise tests with overloaded server can be flaky. +while true +do + $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; + " | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue + break +done + +while true +do + $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; + " | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue + break +done + backup_path="$database_name" for i in $(seq 1 3); do $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" @@ -45,4 +57,4 @@ for i in $(seq 1 3); do $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" done -$CLICKHOUSE_CLIENT -q "DROP DATABASE $database_name SYNC;" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP DATABASE $database_name SYNC;" From dfa5abc754bbbbd46c6cfc5db3ab76d38ef121e1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 23:22:49 +0200 Subject: [PATCH 292/321] Update 02231_buffer_aggregate_states_leak.sql --- .../queries/0_stateless/02231_buffer_aggregate_states_leak.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql index 85915e3790e..5e6b4393a79 100644 --- a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql +++ b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql @@ -29,7 +29,7 @@ from in_02231 group by key; set optimize_trivial_insert_select = 1; -insert into in_02231 select * from numbers(5e6) settings max_memory_usage='30Mi', max_threads=1, min_insert_block_size_rows = 10000, min_insert_block_size_bytes = '10Mi'; +insert into in_02231 select * from numbers(5e6) settings max_memory_usage='40Mi', max_threads=1, min_insert_block_size_rows = 10000, min_insert_block_size_bytes = '10Mi'; drop table buffer_02231; drop table out_02231; From e8c2e1745ee80db01ce164e472c3b0fac31a65b7 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 28 Jul 2024 00:05:24 +0200 Subject: [PATCH 293/321] Fix typo --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index ee253e46014..37541dd4818 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -496,7 +496,7 @@ toInt16OrDefault(expr, def) **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). +- `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int16](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. From be22d76942d7306bbe125b80ac8397e788dac65a Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 28 Jul 2024 00:38:07 +0200 Subject: [PATCH 294/321] add UInt* functions --- .../functions/type-conversion-functions.md | 1484 ++++++++++++++++- 1 file changed, 1466 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 61e84ca72d1..bb386092c0f 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -148,50 +148,1498 @@ Result: └─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ ``` +# toUInt8 -## toUInt(8\|16\|32\|64\|256) +Converts an input value to a value of type [`UInt8`](../data-types/int-uint.md). Throws an exception in case of an error. -Converts an input value to the [UInt](../data-types/int-uint.md) data type. This function family includes: +**Syntax** -- `toUInt8(expr)` — Converts to a value of data type `UInt8`. -- `toUInt16(expr)` — Converts to a value of data type `UInt16`. -- `toUInt32(expr)` — Converts to a value of data type `UInt32`. -- `toUInt64(expr)` — Converts to a value of data type `UInt64`. -- `toUInt256(expr)` — Converts to a value of data type `UInt256`. +```sql +toUInt8(expr) +``` **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +For example: `SELECT toUInt8(256) == 0;`. +::: **Returned value** -- Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type. +- 8-bit unsigned integer value. [UInt8](../data-types/int-uint.md). -Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: -The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. +**Example** + +Query: + +```sql +SELECT + toUInt8(8), + toUInt8(8.8), + toUInt8('8') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt8(8): 8 +toUInt8(8.8): 8 +toUInt8('8'): 8 +``` + +**See also** + +- [`toUInt8OrZero`](#touint8orzero). +- [`toUInt8OrNull`](#touint8ornull). +- [`toUInt8OrDefault`](#touint8ordefault). + +## toUInt8OrZero + +Like [`toUInt8`](#touint8), this function converts an input value to a value of type [UInt8](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt8OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of ordinary Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 8-bit unsigned integer value if successful, otherwise `0`. [UInt8](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: **Example** Query: ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); +SELECT + toUInt8OrZero('-8'), + toUInt8OrZero('abc') +FORMAT vertical; ``` Result: ```response -┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ -│ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ -└─────────────────────┴───────────────┴────────────────┴──────────────┘ +Row 1: +────── +toUInt8OrZero('-8'): 0 +toUInt8OrZero('abc'): 0 ``` -## toUInt(8\|16\|32\|64\|256)OrZero +**See also** -## toUInt(8\|16\|32\|64\|256)OrNull +- [`toUInt8`](#touint8). +- [`toUInt8OrNull`](#touint8ornull). +- [`toUInt8OrDefault`](#touint8ordefault). -## toUInt(8\|16\|32\|64\|256)OrDefault +## toUInt8OrNull + +Like [`toUInt8`](#touint8), this function converts an input value to a value of type [UInt8](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt8OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 8-bit unsigned integer value if successful, otherwise `NULL`. [UInt8](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt8OrNull('-8'), + toUInt8OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt8OrNull('-8'): ᴺᵁᴸᴸ +toUInt8OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt8`](#touint8). +- [`toUInt8OrZero`](#touint8orzero). +- [`toUInt8OrDefault`](#touint8ordefault). + +## toUInt8OrDefault + +Like [`toUInt8`](#touint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`UInt8`](../data-types/int-uint.md). If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toUInt8OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt8` is unsuccessful. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8OrDefault('0xc0fe', CAST('-1', 'UInt8'));`. + +:::note +If the input value cannot be represented within the bounds of [UInt8](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 8-bit unsigned integer value if successful, otherwise returns the default value. [UInt8](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt8OrDefault('8', CAST('0', 'UInt8')), + toUInt8OrDefault('abc', CAST('0', 'UInt8')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt8OrDefault('8', CAST('0', 'UInt8')): 8 +toUInt8OrDefault('abc', CAST('0', 'UInt8')): 0 +``` + +**See also** + +- [`toUInt8`](#touint8). +- [`toUInt8OrZero`](#touint8orzero). +- [`toUInt8OrNull`](#touint8orNull). + +## toUInt16 + +Converts an input value to a value of type [`UInt16`](../data-types/int-uint.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toUInt16(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt16('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +For example: `SELECT toUInt16(65536) == 0;`. +::: + +**Returned value** + +- 16-bit unsigned integer value. [UInt16](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +```sql +SELECT + toUInt16(16), + toUInt16(16.16), + toUInt16('16') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt16(16): 16 +toUInt16(16.16): 16 +toUInt16('16'): 16 +``` + +**See also** + +- [`toUInt16OrZero`](#touint16orzero). +- [`toUInt16OrNull`](#touint16ornull). +- [`toUInt16OrDefault`](#touint16ordefault). + +## toUInt16OrZero + +Like [`toUInt16`](#touint16), this function converts an input value to a value of type [UInt16](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt16OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt16OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered as an error. +::: + +**Returned value** + +- 16-bit unsigned integer value if successful, otherwise `0`. [UInt16](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt16OrZero('16'), + toUInt16OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt16OrZero('16'): 16 +toUInt16OrZero('abc'): 0 +``` + +**See also** + +- [`toUInt16`](#touint16). +- [`toUInt16OrNull`](#touint16ornull). +- [`toUInt16OrDefault`](#touint16ordefault). + +## toUInt16OrNull + +Like [`toUInt16`](#touint16), this function converts an input value to a value of type [UInt16](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt16OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt16OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 16-bit unsigned integer value if successful, otherwise `NULL`. [UInt16](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt16OrNull('16'), + toUInt16OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt16OrNull('16'): 16 +toUInt16OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt16`](#touint16). +- [`toUInt16OrZero`](#touint16orzero). +- [`toUInt16OrDefault`](#touint16ordefault). + +## toUInt16OrDefault + +Like [`toUInt16`](#touint16), this function converts an input value to a value of type [UInt16](../data-types/int-uint.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toInt16OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt16` is unsuccessful. [UInt16](../data-types/int-uint.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt16OrDefault('0xc0fe', CAST('0', 'UInt16'));`. + +:::note +If the input value cannot be represented within the bounds of [UInt16](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 16-bit unsigned integer value if successful, otherwise returns the default value. [UInt16](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. + ::: + +**Example** + +Query: + +``` sql +SELECT + toUInt16OrDefault('16', CAST('0', 'UInt16')), + toUInt16OrDefault('abc', CAST('0', 'UInt16')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt16OrDefault('16', CAST('0', 'UInt16')): 16 +toUInt16OrDefault('abc', CAST('0', 'UInt16')): 0 +``` + +**See also** + +- [`toUInt16`](#touint16). +- [`toUInt16OrZero`](#touint16orzero). +- [`toUInt16OrNull`](#touint16ornull). + +## toUInt32 + +Converts an input value to a value of type [`UInt32`](../data-types/int-uint.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toUInt32(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt32('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt32](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +For example: `SELECT toUInt32(4294967296) == 0;` +::: + +**Returned value** + +- 32-bit unsigned integer value. [Int32](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +```sql +SELECT + toUInt32(32), + toUInt32(32.32), + toUInt32('32') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt32(32): 32 +toUInt32(32.32): 32 +toUInt32('32'): 32 +``` + +**See also** + +- [`toUInt32OrZero`](#touint32orzero). +- [`toUInt32OrNull`](#touint32ornull). +- [`toUInt32OrDefault`](#touint32ordefault). + +## toUInt32OrZero + +Like [`toUInt32`](#touint32), this function converts an input value to a value of type [UInt32](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt32OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt32OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 32-bit unsigned integer value if successful, otherwise `0`. [UInt32](../data-types/int-uint.md) + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero) +, meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt32OrZero('32'), + toUInt32OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt32OrZero('32'): 32 +toUInt32OrZero('abc'): 0 +``` +**See also** + +- [`toUInt32`](#touint32). +- [`toUInt32OrNull`](#touint32ornull). +- [`toUInt32OrDefault`](#touint32ordefault). + +## toUInt32OrNull + +Like [`toUInt32`](#touint32), this function converts an input value to a value of type [UInt32](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt32OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt32OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 32-bit unsigned integer value if successful, otherwise `NULL`. [UInt32](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero) +, meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt32OrNull('32'), + toUInt32OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt32OrNull('32'): 32 +toUInt32OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt32`](#touint32). +- [`toUInt32OrZero`](#touint32orzero). +- [`toUInt32OrDefault`](#touint32ordefault). + +## toUInt32OrDefault + +Like [`toUInt32`](#touint32), this function converts an input value to a value of type [UInt32](../data-types/int-uint.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toUInt32OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt32` is unsuccessful. [UInt32](../data-types/int-uint.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt32OrDefault('0xc0fe', CAST('0', 'UInt32'));`. + +:::note +If the input value cannot be represented within the bounds of [UInt32](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 32-bit unsigned integer value if successful, otherwise returns the default value. [UInt32](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt32OrDefault('32', CAST('0', 'UInt32')), + toUInt32OrDefault('abc', CAST('0', 'UInt32')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt32OrDefault('32', CAST('0', 'UInt32')): 32 +toUInt32OrDefault('abc', CAST('0', 'UInt32')): 0 +``` + +**See also** + +- [`toUInt32`](#touint32). +- [`toUInt32OrZero`](#touint32orzero). +- [`toUInt32OrNull`](#touint32ornull). + +## toUInt64 + +Converts an input value to a value of type [`UInt64`](../data-types/int-uint.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toUInt64(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported types: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt64('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt64](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +For example: `SELECT toUInt64(18446744073709551616) == 0;` +::: + +**Returned value** + +- 64-bit unsigned integer value. [UInt64](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +```sql +SELECT + toUInt64(64), + toUInt64(64.64), + toUInt64('64') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt64(64): 64 +toUInt64(64.64): 64 +toUInt64('64'): 64 +``` + +**See also** + +- [`toUInt64OrZero`](#touint64orzero). +- [`toUInt64OrNull`](#touint64ornull). +- [`toUInt64OrDefault`](#touint64ordefault). + +## toInt64OrZero + +Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt64OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt64OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 64-bit unsigned integer value if successful, otherwise `0`. [UInt64](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt64OrZero('64'), + toUInt64OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt64OrZero('64'): 64 +toUInt64OrZero('abc'): 0 +``` + +**See also** + +- [`toUInt64`](#touint64). +- [`toUInt64OrNull`](#touint64ornull). +- [`toUInt64OrDefault`](#touint64ordefault). + +## toUInt64OrNull + +Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt64OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt64OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 64-bit unsigned integer value if successful, otherwise `NULL`. [UInt64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt64OrNull('64'), + toUInt64OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt64OrNull('64'): 64 +toUInt64OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt64`](#touint64). +- [`toUInt64OrZero`](#touint64orzero). +- [`toUInt64OrDefault`](#touint64ordefault). + +## toUInt64OrDefault + +Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toUInt64OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt64` is unsuccessful. [UInt64](../data-types/int-uint.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt64OrDefault('0xc0fe', CAST('0', 'UInt64'));`. + +:::note +If the input value cannot be represented within the bounds of [UInt64](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 64-bit unsigned integer value if successful, otherwise returns the default value. [UInt64](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt64OrDefault('64', CAST('0', 'UInt64')), + toUInt64OrDefault('abc', CAST('0', 'UInt64')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt64OrDefault('64', CAST('0', 'UInt64')): 64 +toUInt64OrDefault('abc', CAST('0', 'UInt64')): 0 +``` + +**See also** + +- [`toUInt64`](#touint64). +- [`toUInt64OrZero`](#touint64orzero). +- [`toUInt64OrNull`](#touint64ornull). + +## toUInt128 + +Converts an input value to a value of type [`UInt128`](../data-types/int-uint.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toUInt128(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt128('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt128](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +::: + +**Returned value** + +- 128-bit unsigned integer value. [UInt128](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +```sql +SELECT + toUInt128(128), + toUInt128(128.8), + toUInt128('128') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt128(128): 128 +toUInt128(128.8): 128 +toUInt128('128'): 128 +``` + +**See also** + +- [`toUInt128OrZero`](#touint128orzero). +- [`toUInt128OrNull`](#touint128ornull). +- [`toUInt128OrDefault`](#touint128ordefault). + +## toUInt128OrZero + +Like [`toUInt128`](#touint128), this function converts an input value to a value of type [UInt128](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt128OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt128OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 128-bit unsigned integer value if successful, otherwise `0`. [UInt128](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt128OrZero('128'), + toUInt128OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt128OrZero('128'): 128 +toUInt128OrZero('abc'): 0 +``` + +**See also** + +- [`toUInt128`](#touint128). +- [`toUInt128OrNull`](#touint128ornull). +- [`toUInt128OrDefault`](#touint128ordefault). + +## toUInt128OrNull + +Like [`toUInt128`](#touint128), this function converts an input value to a value of type [UInt128](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt128OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt128OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 128-bit unsigned integer value if successful, otherwise `NULL`. [UInt128](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt128OrNull('128'), + toUInt128OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt128OrNull('128'): 128 +toUInt128OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt128`](#touint128). +- [`toUInt128OrZero`](#touint128orzero). +- [`toUInt128OrDefault`](#touint128ordefault). + +## toUInt128OrDefault + +Like [`toUInt128`](#toint128), this function converts an input value to a value of type [UInt128](../data-types/int-uint.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toUInt128OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt128` is unsuccessful. [UInt128](../data-types/int-uint.md). + +Supported arguments: +- (U)Int8/16/32/64/128/256. +- Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt128OrDefault('0xc0fe', CAST('0', 'UInt128'));`. + +:::note +If the input value cannot be represented within the bounds of [UInt128](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 128-bit unsigned integer value if successful, otherwise returns the default value. [UInt128](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt128OrDefault('128', CAST('0', 'UInt128')), + toUInt128OrDefault('abc', CAST('0', 'UInt128')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt128OrDefault('128', CAST('0', 'UInt128')): 128 +toUInt128OrDefault('abc', CAST('0', 'UInt128')): 0 +``` + +**See also** + +- [`toUInt128`](#touint128). +- [`toUInt128OrZero`](#touint128orzero). +- [`toUInt128OrNull`](#touint128ornull). + +## toUInt256 + +Converts an input value to a value of type [`UInt256`](../data-types/int-uint.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toUInt256(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments: +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt256('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt256](../data-types/int-uint.md), the result over or under flows. +This is not considered an error. +::: + +**Returned value** + +- 256-bit unsigned integer value. [Int256](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +```sql +SELECT + toUInt256(256), + toUInt256(256.256), + toUInt256('256') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt256(256): 256 +toUInt256(256.256): 256 +toUInt256('256'): 256 +``` + +**See also** + +- [`toUInt256OrZero`](#touint256orzero). +- [`toUInt256OrNull`](#touint256ornull). +- [`toUInt256OrDefault`](#touint256ordefault). + +## toUUInt256OrZero + +Like [`toUInt256`](#touint256), this function converts an input value to a value of type [UInt256](../data-types/int-uint.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toUInt256OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `0`): +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt256OrZero('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 256-bit unsigned integer value if successful, otherwise `0`. [UInt256](../data-types/int-uint.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt256OrZero('256'), + toUInt256OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt256OrZero('256'): 256 +toUInt256OrZero('abc'): 0 +``` + +**See also** + +- [`toUInt256`](#touint256). +- [`toUInt256OrNull`](#touint256ornull). +- [`toUInt256OrDefault`](#touint256ordefault). + +## toUInt256OrNull + +Like [`toUInt256`](#touint256), this function converts an input value to a value of type [UInt256](../data-types/int-uint.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toUInt256OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256. + +Unsupported arguments (return `\N`) +- String representations of Float32/64 values, including `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt256OrNull('0xc0fe');`. + +:::note +If the input value cannot be represented within the bounds of [UInt256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 256-bit unsigned integer value if successful, otherwise `NULL`. [UInt256](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt256OrNull('256'), + toUInt256OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt256OrNull('256'): 256 +toUInt256OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toUInt256`](#touint256). +- [`toUInt256OrZero`](#touint256orzero). +- [`toUInt256OrDefault`](#touint256ordefault). + +## toUInt256OrDefault + +Like [`toUInt256`](#touint256), this function converts an input value to a value of type [UInt256](../data-types/int-uint.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toUInt256OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `UInt256` is unsuccessful. [UInt256](../data-types/int-uint.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- String representations of (U)Int8/16/32/128/256. + +Arguments for which the default value is returned: +- String representations of Float32/64 values, including `NaN` and `Inf` +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt256OrDefault('0xc0fe', CAST('0', 'UInt256'));` + +:::note +If the input value cannot be represented within the bounds of [UInt256](../data-types/int-uint.md), overflow or underflow of the result occurs. +This is not considered an error. +::: + +**Returned value** + +- 256-bit unsigned integer value if successful, otherwise returns the default value. [UInt256](../data-types/int-uint.md). + +:::note +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +**Example** + +Query: + +``` sql +SELECT + toUInt256OrDefault('-256', CAST('0', 'UInt256')), + toUInt256OrDefault('abc', CAST('0', 'UInt256')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toUInt256OrDefault('-256', CAST('0', 'UInt256')): 0 +toUInt256OrDefault('abc', CAST('0', 'UInt256')): 0 +``` + +**See also** + +- [`toUInt256`](#touint256). +- [`toUInt256OrZero`](#touint256orzero). +- [`toUInt256OrNull`](#touint256ornull). ## toFloat(32\|64) From 96eee7469a342648bce942c93ca0c5673234a662 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 13:31:44 +0200 Subject: [PATCH 295/321] Update 02231_buffer_aggregate_states_leak.sql --- .../queries/0_stateless/02231_buffer_aggregate_states_leak.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql index 5e6b4393a79..7cc267b7263 100644 --- a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql +++ b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql @@ -29,7 +29,7 @@ from in_02231 group by key; set optimize_trivial_insert_select = 1; -insert into in_02231 select * from numbers(5e6) settings max_memory_usage='40Mi', max_threads=1, min_insert_block_size_rows = 10000, min_insert_block_size_bytes = '10Mi'; +insert into in_02231 select * from numbers(5e6) settings max_memory_usage='400Mi', max_threads=1; drop table buffer_02231; drop table out_02231; From 28f7eef7a96837dca1ec72fbcfffcfbb7ca965e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 13:37:26 +0200 Subject: [PATCH 296/321] Double whitespace --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index cee7de3b1db..0c4d6988e71 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -34,7 +34,7 @@ verify() } $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS test_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table (data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" $CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" verify From 22211e30aa5762a499ad42e40f565690fb283e5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 13:42:45 +0200 Subject: [PATCH 297/321] Revert "Enable `compile_expressions` by default." --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 3 +-- tests/clickhouse-test | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c7a1a7e2739..86e04b2ab4e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -186,7 +186,7 @@ class IColumn; M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \ M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \ - M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 87eaeff0ca9..9faf77e9087 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.8", {{"compile_expressions", false, true, "We believe that the LLVM infrastructure behind the JIT compiler is stable enough to enable this setting by default."}}}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, @@ -81,7 +80,7 @@ static std::initializer_list Date: Sun, 28 Jul 2024 13:55:23 +0200 Subject: [PATCH 298/321] Add documentation for `compile_expressions` --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86e04b2ab4e..3349662cb78 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -186,7 +186,7 @@ class IColumn; M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \ M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \ - M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code. Due to a bug in the LLVM compiler infrastructure, on AArch64 machines, it is known to lead to a nullptr dereference and, consequently, server crash. Do not enable this setting.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ From 29b804adf38401dd47298ed285aee587b732595b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 14:01:17 +0200 Subject: [PATCH 299/321] Wrap log lines in the CI report for functional tests --- tests/ci/report.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/report.py b/tests/ci/report.py index 77043dfc8be..48e24f9c870 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -135,6 +135,7 @@ th {{ cursor: pointer; }} tr:hover {{ filter: var(--tr-hover-filter); }} .expandable {{ cursor: pointer; }} .expandable-content {{ display: none; }} +pre {{ white-space: pre-wrap; }} #fish {{ display: none; float: right; position: relative; top: -20em; right: 2vw; margin-bottom: -20em; width: 30vw; filter: brightness(7%); z-index: -1; }} .themes {{ From 4192bcf2b3e9a144756aea204d8dd5ad7036925e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 15:59:35 +0200 Subject: [PATCH 300/321] Update 01600_parts_states_metrics_long.sh --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index fff98a9dd20..fa01ca4fbef 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From fd4a733d647b094bb4cf07ed914484d8cd656d61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 18:01:18 +0200 Subject: [PATCH 301/321] Fix bad test `02050_client_profile_events` --- .../queries/0_stateless/02050_client_profile_events.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 05e48de771d..e20590a6d45 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -21,11 +21,17 @@ echo 'regression test for overlap profile events snapshots between queries (clic $CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'print everything' -profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" +profile_events="$( + $CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& + $CLICKHOUSE_LOCAL --input-format LineAsString --query "SELECT sum(extract(line, 'SelectedRows: (\\d+)')::UInt64) FROM table WHERE line LIKE '%SelectedRows: %'" +)" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'print each 100 ms' -profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events --profile-events-delay-ms=100 -q 'select sleep(0.2) from numbers(10) format Null' |& grep -c 'SelectedRows')" +profile_events="$( + $CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events --profile-events-delay-ms=100 -q 'select sleep(0.2) from numbers(10) format Null' |& + $CLICKHOUSE_LOCAL --input-format LineAsString --query "SELECT sum(extract(line, 'SelectedRows: (\\d+)')::UInt64) FROM table WHERE line LIKE '%SelectedRows: %'" +)" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'check that ProfileEvents is new for each query' From f2b8431867e3f1147d3ed942a3723e9389d3bac2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 20:23:00 +0200 Subject: [PATCH 302/321] I do not want to think about this code --- src/Storages/MergeTree/MergeTreeData.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 30a4a7caa0f..e5116afa3fb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4405,25 +4405,25 @@ bool MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) size_t MergeTreeData::getTotalActiveSizeInBytes() const { - return total_active_size_bytes.load(std::memory_order_acquire); + return total_active_size_bytes.load(); } size_t MergeTreeData::getTotalActiveSizeInRows() const { - return total_active_size_rows.load(std::memory_order_acquire); + return total_active_size_rows.load(); } size_t MergeTreeData::getActivePartsCount() const { - return total_active_size_parts.load(std::memory_order_acquire); + return total_active_size_parts.load(); } size_t MergeTreeData::getOutdatedPartsCount() const { - return total_outdated_parts_count.load(std::memory_order_relaxed); + return total_outdated_parts_count.load(); } size_t MergeTreeData::getNumberOfOutdatedPartsWithExpiredRemovalTime() const @@ -8184,16 +8184,16 @@ void MergeTreeData::removePartContributionToDataVolume(const DataPartPtr & part) void MergeTreeData::increaseDataVolume(ssize_t bytes, ssize_t rows, ssize_t parts) { - total_active_size_bytes.fetch_add(bytes, std::memory_order_acq_rel); - total_active_size_rows.fetch_add(rows, std::memory_order_acq_rel); - total_active_size_parts.fetch_add(parts, std::memory_order_acq_rel); + total_active_size_bytes.fetch_add(bytes); + total_active_size_rows.fetch_add(rows); + total_active_size_parts.fetch_add(parts); } void MergeTreeData::setDataVolume(size_t bytes, size_t rows, size_t parts) { - total_active_size_bytes.store(bytes, std::memory_order_release); - total_active_size_rows.store(rows, std::memory_order_release); - total_active_size_parts.store(parts, std::memory_order_release); + total_active_size_bytes.store(bytes); + total_active_size_rows.store(rows); + total_active_size_parts.store(parts); } bool MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const From ce1c965b297c7ea88535cbd53501790bbf1fdbd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 20:56:51 +0200 Subject: [PATCH 303/321] Fix test `00940_max_parts_in_total` --- tests/queries/0_stateless/00940_max_parts_in_total.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00940_max_parts_in_total.sql b/tests/queries/0_stateless/00940_max_parts_in_total.sql index e2633c2f0f8..955f593c907 100644 --- a/tests/queries/0_stateless/00940_max_parts_in_total.sql +++ b/tests/queries/0_stateless/00940_max_parts_in_total.sql @@ -1,6 +1,7 @@ drop table if exists max_parts_in_total; create table max_parts_in_total (x UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x SETTINGS max_parts_in_total = 10; +SET max_insert_threads = 1; INSERT INTO max_parts_in_total SELECT number FROM numbers(10); SELECT 1; INSERT INTO max_parts_in_total SELECT 123; -- { serverError TOO_MANY_PARTS } From 10b3cb92bc0294d12da77a9aac40e56dd1b8f6ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 21:14:47 +0200 Subject: [PATCH 304/321] Tests for Kafka cannot run in parallel --- tests/integration/parallel_skip.json | 70 ++++++++++++++++++++++++++-- 1 file changed, 67 insertions(+), 3 deletions(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 3c3d1b6cc96..75b8d25ec04 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -43,7 +43,6 @@ "test_replicated_database/test.py::test_startup_without_zk", "test_replicated_database/test.py::test_sync_replica", "test_replicated_fetches_timeouts/test.py::test_no_stall", - "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", "test_storage_s3/test.py::test_url_reconnect_in_the_middle", "test_system_metrics/test.py::test_readonly_metrics", "test_system_replicated_fetches/test.py::test_system_replicated_fetches", @@ -73,7 +72,7 @@ "test_keeper_map/test.py::test_keeper_map_without_zk", "test_replicated_merge_tree_wait_on_shutdown/test.py::test_shutdown_and_wait", - + "test_http_failover/test.py::test_url_destination_host_with_multiple_addrs", "test_http_failover/test.py::test_url_invalid_hostname", "test_http_failover/test.py::test_url_ip_change", @@ -97,5 +96,70 @@ "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_stop_moves_query", "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_table_detach", - "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_zookeeper_disconnect" + "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_zookeeper_disconnect", + + "test_storage_kafka/test.py::test_kafka_column_types", + "test_storage_kafka/test.py::test_kafka_settings_old_syntax", + "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_settings_predefined_macros", + "test_storage_kafka/test.py::test_kafka_json_as_string", + "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_issue11308", + "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_consumer_hang", + "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_read_consumers_in_parallel", + "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_select_empty", + "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_protobuf", + "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", + "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_kafka_recreate_kafka_table", + "test_storage_kafka/test.py::test_librdkafka_compression", + "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_many_materialized_views", + "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_virtual_columns", + "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_kafka_insert", + "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_commit_on_block_write", + "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_producer_consumer_separate_settings", + "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", + "test_storage_kafka/test.py::test_kafka_insert_avro", + "test_storage_kafka/test.py::test_kafka_produce_consume_avro", + "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_by_block_size", + "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_rebalance", + "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_exception_from_destructor", + "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_bad_reschedule", + "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_premature_flush_on_eof", + "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_issue14202", + "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", + "test_storage_kafka/test.py::test_kafka_consumer_failover", + "test_storage_kafka/test.py::test_kafka_predefined_configuration", + "test_storage_kafka/test.py::test_issue26643", + "test_storage_kafka/test.py::test_num_consumers_limit", + "test_storage_kafka/test.py::test_format_with_prefix_and_suffix", + "test_storage_kafka/test.py::test_max_rows_per_message", + "test_storage_kafka/test.py::test_row_based_formats", + "test_storage_kafka/test.py::test_block_based_formats_1", + "test_storage_kafka/test.py::test_block_based_formats_2", + "test_storage_kafka/test.py::test_system_kafka_consumers", + "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance", + "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv", + "test_storage_kafka/test.py::test_formats_errors", + "test_storage_kafka/test.py::test_multiple_read_in_materialized_views", ] From 1f784d1862d7b2c60ea890019d8663f8ef7858b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 21:15:23 +0200 Subject: [PATCH 305/321] Tests for Kafka cannot run in parallel --- tests/integration/parallel_skip.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 75b8d25ec04..9b970a326c4 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -161,5 +161,5 @@ "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance", "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv", "test_storage_kafka/test.py::test_formats_errors", - "test_storage_kafka/test.py::test_multiple_read_in_materialized_views", + "test_storage_kafka/test.py::test_multiple_read_in_materialized_views" ] From 8115bb7a54e91b993e9d287bb049f128fea489f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 01:01:03 +0200 Subject: [PATCH 306/321] Update 01600_parts_states_metrics_long.sh --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index fa01ca4fbef..7215f270a4c 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -16,13 +16,13 @@ verify_sql="SELECT # In case of test failure, this code will do infinite loop and timeout. verify() { - for i in {1..100} + for i in {1..5000} do result=$( $CLICKHOUSE_CLIENT --query="$verify_sql" ) [ "$result" = "1" ] && echo "$result" && break sleep 0.1 - if [[ $i -eq 100 ]] + if [[ $i -eq 5000 ]] then $CLICKHOUSE_CLIENT --multiquery " SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; From 6e31e813b87aafe4b97b18c5cbbce9483402bde1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 01:45:29 +0200 Subject: [PATCH 307/321] Add a test for #66285 --- tests/queries/0_stateless/03213_distributed_analyzer.reference | 1 + tests/queries/0_stateless/03213_distributed_analyzer.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03213_distributed_analyzer.reference create mode 100644 tests/queries/0_stateless/03213_distributed_analyzer.sql diff --git a/tests/queries/0_stateless/03213_distributed_analyzer.reference b/tests/queries/0_stateless/03213_distributed_analyzer.reference new file mode 100644 index 00000000000..9d63c0a7a5e --- /dev/null +++ b/tests/queries/0_stateless/03213_distributed_analyzer.reference @@ -0,0 +1 @@ +['digraph','{',' rankdir="LR";',' { node [shape = rect]',' n1 [label="Remote"];',' }','}','digraph','{',' rankdir="LR";',' { node [shape = rect]',' n1 [label="Remote"];',' }','}'] diff --git a/tests/queries/0_stateless/03213_distributed_analyzer.sql b/tests/queries/0_stateless/03213_distributed_analyzer.sql new file mode 100644 index 00000000000..ae0f13f08f0 --- /dev/null +++ b/tests/queries/0_stateless/03213_distributed_analyzer.sql @@ -0,0 +1,3 @@ +-- This triggered a nullptr dereference due to the confusion between old and new analyzers: +SELECT sum(*) FROM remote('127.0.0.4', currentDatabase(), viewExplain('EXPLAIN PIPELINE', 'graph = 1', (SELECT * FROM remote('127.0.0.4', system, one)))); -- { serverError UNKNOWN_FUNCTION } +SELECT groupArray(*) FROM cluster(test_cluster_two_shards, viewExplain('EXPLAIN PIPELINE', 'graph = 1', (SELECT * FROM remote('127.0.0.4', system, one)))); From a1ab412fcbbf7867ac833cf1cc0261c65b5a3195 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 02:00:30 +0200 Subject: [PATCH 308/321] Prevent misuse of random number distributions --- src/Functions/randDistribution.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Functions/randDistribution.cpp b/src/Functions/randDistribution.cpp index 6a3dac748c1..dc2ddf2ef24 100644 --- a/src/Functions/randDistribution.cpp +++ b/src/Functions/randDistribution.cpp @@ -93,6 +93,9 @@ struct ChiSquaredDistribution static void generate(Float64 degree_of_freedom, ColumnFloat64::Container & container) { + if (degree_of_freedom <= 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName()); + auto distribution = std::chi_squared_distribution<>(degree_of_freedom); for (auto & elem : container) elem = distribution(thread_local_rng); @@ -107,6 +110,9 @@ struct StudentTDistribution static void generate(Float64 degree_of_freedom, ColumnFloat64::Container & container) { + if (degree_of_freedom <= 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName()); + auto distribution = std::student_t_distribution<>(degree_of_freedom); for (auto & elem : container) elem = distribution(thread_local_rng); @@ -121,6 +127,9 @@ struct FisherFDistribution static void generate(Float64 d1, Float64 d2, ColumnFloat64::Container & container) { + if (d1 <= 0 || d2 <= 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument (degrees of freedom) of function {} should be greater than zero", getName()); + auto distribution = std::fisher_f_distribution<>(d1, d2); for (auto & elem : container) elem = distribution(thread_local_rng); @@ -300,7 +309,7 @@ public: } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than two argument specified for function {}", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than two arguments specified for function {}", getName()); } return res_column; From 5216a3d4f260e770b8bf780fbba11dfe822bbd4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 02:05:25 +0200 Subject: [PATCH 309/321] Add a test --- tests/queries/0_stateless/03213_rand_dos.reference | 0 tests/queries/0_stateless/03213_rand_dos.sql | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03213_rand_dos.reference create mode 100644 tests/queries/0_stateless/03213_rand_dos.sql diff --git a/tests/queries/0_stateless/03213_rand_dos.reference b/tests/queries/0_stateless/03213_rand_dos.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03213_rand_dos.sql b/tests/queries/0_stateless/03213_rand_dos.sql new file mode 100644 index 00000000000..1250995bfe5 --- /dev/null +++ b/tests/queries/0_stateless/03213_rand_dos.sql @@ -0,0 +1,5 @@ +SELECT randChiSquared(-0.0000001); -- { serverError BAD_ARGUMENTS } +SELECT randChiSquared(-0.0); -- { serverError BAD_ARGUMENTS } +SELECT randStudentT(-0.); -- { serverError BAD_ARGUMENTS } +SELECT randFisherF(-0., 1); -- { serverError BAD_ARGUMENTS } +SELECT randFisherF(1, -0.); -- { serverError BAD_ARGUMENTS } From 39de59a605ffe6ad63990a219b49781816c4aaa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 02:11:55 +0200 Subject: [PATCH 310/321] Follow-up for #67301 --- tests/ci/report.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 48e24f9c870..f50ed4c1f85 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -125,7 +125,7 @@ html {{ min-height: 100%; font-family: "DejaVu Sans", "Noto Sans", Arial, sans-s h1 {{ margin-left: 10px; }} th, td {{ padding: 5px 10px 5px 10px; text-align: left; vertical-align: top; line-height: 1.5; border: 1px solid var(--table-border-color); }} td {{ background: var(--td-background); }} -th {{ background: var(--th-background); }} +th {{ background: var(--th-background); white-space: nowrap; }} a {{ color: var(--link-color); text-decoration: none; }} a:hover, a:active {{ color: var(--link-hover-color); text-decoration: none; }} table {{ box-shadow: 0 8px 25px -5px rgba(0, 0, 0, var(--shadow-intensity)); border-collapse: collapse; border-spacing: 0; }} From e274aca937bb1b89e6c8646457b046a827093ecb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 02:40:33 +0200 Subject: [PATCH 311/321] Add a test for #61659 --- tests/queries/0_stateless/03213_array_element_msan.reference | 5 +++++ tests/queries/0_stateless/03213_array_element_msan.sql | 2 ++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03213_array_element_msan.reference create mode 100644 tests/queries/0_stateless/03213_array_element_msan.sql diff --git a/tests/queries/0_stateless/03213_array_element_msan.reference b/tests/queries/0_stateless/03213_array_element_msan.reference new file mode 100644 index 00000000000..6844e39d0aa --- /dev/null +++ b/tests/queries/0_stateless/03213_array_element_msan.reference @@ -0,0 +1,5 @@ +10 +-- system.settings_profiles +-- system.settings_profiles +-- system.settings_profiles +-- system.settings_profiles diff --git a/tests/queries/0_stateless/03213_array_element_msan.sql b/tests/queries/0_stateless/03213_array_element_msan.sql new file mode 100644 index 00000000000..68955856792 --- /dev/null +++ b/tests/queries/0_stateless/03213_array_element_msan.sql @@ -0,0 +1,2 @@ +SELECT [[10, 2, 13, 15][toNullable(toLowCardinality(1))]][materialize(toLowCardinality(1))]; +SELECT '-- system.settings_profiles' GROUP BY [[[[[[[[[[10, toNullable(10)][1], [materialize(toLowCardinality(10)), 2][materialize(toLowCardinality(1))]][1]][materialize(materialize(1))], [10, 2, 1][1]][1]][1], 1][toLowCardinality(1)]][1], 1][1], 10][1], [[10, toLowCardinality(2)][toNullable(toLowCardinality(1))]][materialize(toLowCardinality(1))]][1], [[[[10, 2][1]][1]][1], [10, 2][materialize(1)], [[[2][1]][materialize(1)], 2, 1][1], [2, 10, toNullable(1)][1]] WITH CUBE; From bdcedeb3f0fba764c7471bc4a95a6fafdfaeaf7e Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 29 Jul 2024 09:49:18 +0200 Subject: [PATCH 312/321] CI: Add stdout, err for docker pull command --- tests/ci/ci_utils.py | 3 ++- tests/ci/docker_images_helper.py | 24 +++++++----------------- 2 files changed, 9 insertions(+), 18 deletions(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index d42091fb0da..447aac74c7f 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -212,7 +212,7 @@ class Shell: return res.stdout.strip() @classmethod - def run(cls, command, check=False, dry_run=False): + def run(cls, command, check=False, dry_run=False, **kwargs): if dry_run: print(f"Dry-ryn. Would run command [{command}]") return "" @@ -225,6 +225,7 @@ class Shell: stderr=subprocess.PIPE, text=True, check=False, + **kwargs, ) if result.returncode == 0: print(f"stdout: {result.stdout.strip()}") diff --git a/tests/ci/docker_images_helper.py b/tests/ci/docker_images_helper.py index 6ea679e0597..e6869852c4e 100644 --- a/tests/ci/docker_images_helper.py +++ b/tests/ci/docker_images_helper.py @@ -3,12 +3,12 @@ import json import logging import os -import subprocess from pathlib import Path from typing import Any, Dict, List, Optional from env_helper import ROOT_DIR, DOCKER_TAG from get_robot_token import get_parameter_from_ssm +from ci_utils import Shell IMAGES_FILE_PATH = Path("docker/images.json") @@ -16,20 +16,14 @@ ImagesDict = Dict[str, dict] def docker_login(relogin: bool = True) -> None: - if ( - relogin - or subprocess.run( # pylint: disable=unexpected-keyword-arg - "docker system info | grep --quiet -E 'Username|Registry'", - shell=True, - check=False, - ).returncode - == 1 + if relogin or not Shell.check( + "docker system info | grep --quiet -E 'Username|Registry'" ): - subprocess.check_output( # pylint: disable=unexpected-keyword-arg + Shell.run( # pylint: disable=unexpected-keyword-arg "docker login --username 'robotclickhouse' --password-stdin", input=get_parameter_from_ssm("dockerhub_robot_password"), encoding="utf-8", - shell=True, + check=True, ) @@ -48,14 +42,10 @@ class DockerImage: def pull_image(image: DockerImage) -> DockerImage: try: logging.info("Pulling image %s - start", image) - subprocess.check_output( - f"docker pull {image}", - stderr=subprocess.STDOUT, - shell=True, - ) + Shell.run(f"docker pull {image}", check=True) logging.info("Pulling image %s - done", image) except Exception as ex: - logging.info("Got execption pulling docker %s", ex) + logging.info("Got exception pulling docker %s", ex) raise ex return image From e05972d9a6df1aa8853626e5f83a65da45e9561a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Jul 2024 08:39:18 +0000 Subject: [PATCH 313/321] Followup #67290 --- tests/queries/0_stateless/02911_backup_restore_keeper_map.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index 385583acbbe..ee070b40f6f 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -17,14 +17,14 @@ $CLICKHOUSE_CLIENT -nm -q " while true do $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; - " | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue + " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue break done while true do $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; - " | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue + " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue break done From ac001c0c6fe9bde746ffc4e688b74d81de9c1dc8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 29 Jul 2024 08:41:23 +0000 Subject: [PATCH 314/321] no parallel for kerberized kafka --- tests/integration/parallel_skip.json | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 9b970a326c4..99fa626bd1e 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -161,5 +161,10 @@ "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance", "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv", "test_storage_kafka/test.py::test_formats_errors", - "test_storage_kafka/test.py::test_multiple_read_in_materialized_views" + "test_storage_kafka/test.py::test_multiple_read_in_materialized_views", + + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_request_new_ticket_after_expiration", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection" ] From 906d709d84757220355dd1e7b0b24c793a185f96 Mon Sep 17 00:00:00 2001 From: wxybear Date: Fri, 26 Jul 2024 01:40:37 +0800 Subject: [PATCH 315/321] clickhouse-client/local: make multi-query mode the default --- programs/client/Client.cpp | 14 --- programs/local/LocalServer.cpp | 10 -- src/Client/ClientBase.cpp | 102 ++++++++++-------- src/Client/ClientBase.h | 3 - src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 1 - .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 - .../00366_multi_statements.reference | 6 +- .../0_stateless/00366_multi_statements.sh | 4 +- .../0_stateless/00499_json_enum_insert.sql | 3 + .../0_stateless/00517_date_parsing.sql | 1 + .../00560_float_leading_plus_in_exponent.sql | 2 + .../0_stateless/00700_decimal_formats.sql | 16 +-- .../00760_insert_json_with_defaults.sql | 9 ++ .../0_stateless/00861_decimal_quoted_csv.sql | 9 +- tests/queries/0_stateless/00945_ml_test.sql | 2 +- tests/queries/0_stateless/00946_ml_test.sql | 2 +- .../00976_system_stop_ttl_merges.sql | 2 +- .../0_stateless/01034_JSONCompactEachRow.sql | 13 +++ .../01060_defaults_all_columns.sql | 1 + ..._json_each_row_data_in_square_brackets.sql | 4 + .../01179_insert_values_semicolon.expect | 4 +- .../0_stateless/01231_operator_null_in.sql | 2 +- ..._block_size_rows_for_materialized_views.sh | 2 +- .../0_stateless/01299_alter_merge_tree.sql | 4 +- .../0_stateless/01324_insert_tsv_raw.sql | 1 - ...6_low_cardinality_fixed_string_default.sql | 1 + .../01448_json_compact_strings_each_row.sql | 13 +++ ...nput_format_csv_enum_as_number_setting.sql | 1 + ...01514_input_format_json_enum_as_number.sql | 1 + ...nput_format_tsv_enum_as_number_setting.sql | 2 + .../01558_enum_as_num_in_tsv_csv_input.sql | 8 ++ .../01564_test_hint_woes.reference | 2 +- .../queries/0_stateless/01825_type_json_1.sql | 5 +- .../0_stateless/01825_type_json_10.sql | 1 + .../0_stateless/01825_type_json_17.sql | 4 + .../0_stateless/01825_type_json_18.sql | 2 + .../queries/0_stateless/01825_type_json_9.sql | 1 + .../0_stateless/01825_type_json_in_array.sql | 2 + .../01825_type_json_insert_select.sql | 8 +- ...json_as_string_data_in_square_brackets.sql | 3 + .../0_stateless/01943_query_id_check.sql | 2 +- .../0_stateless/02002_parse_map_int_key.sql | 1 - .../02105_backslash_letter_commands.expect | 1 - tests/queries/0_stateless/02114_bool_type.sql | 9 ++ .../0_stateless/02184_ipv6_select_parsing.sql | 3 + .../0_stateless/02192_comment_error.reference | 4 +- .../0_stateless/02192_comment_error.sh | 2 + .../02193_async_insert_tcp_client_1.sql | 4 +- ...02234_clickhouse_local_test_mode.reference | 2 +- .../02234_clickhouse_local_test_mode.sh | 4 +- .../0_stateless/02242_subcolumns_sizes.sql | 1 + ...ert_select_from_input_schema_inference.sql | 1 + .../02267_jsonlines_ndjson_format.sql | 1 + .../0_stateless/02366_kql_summarize.sql | 2 +- .../02387_parse_date_as_datetime.sql | 6 ++ .../02435_rollback_cancelled_queries.sh | 2 +- ...458_use_structure_from_insertion_table.sql | 11 +- .../02495_analyzer_storage_join.sql | 2 +- .../02504_regexp_dictionary_table_source.sql | 23 ++-- ...e_with_nonconst_needle_and_replacement.sql | 4 +- .../02553_type_json_attach_partition.sql | 1 + .../02726_async_insert_flush_queue.sql | 8 +- .../02751_multiquery_with_argument.reference | 2 - .../02751_multiquery_with_argument.sh | 5 +- .../02771_multiple_query_arguments.reference | 10 +- .../02771_multiple_query_arguments.sh | 8 +- ...2875_clickhouse_local_multiquery.reference | 6 +- .../02875_clickhouse_local_multiquery.sh | 9 +- ...alues_allow_data_after_semicolon.reference | 6 -- ...ormat_values_allow_data_after_semicolon.sh | 11 +- ...900_window_function_with_sparse_column.sql | 2 +- .../02932_refreshable_materialized_views.sh | 3 +- .../02950_dictionary_short_circuit.sql | 12 +-- .../02974_analyzer_array_join_subcolumn.sql | 2 +- .../03005_input_function_in_join.sql | 2 +- .../03156_default_multiquery_split.reference | 10 ++ .../03156_default_multiquery_split.sh | 59 ++++++++++ 78 files changed, 335 insertions(+), 180 deletions(-) create mode 100644 tests/queries/0_stateless/03156_default_multiquery_split.reference create mode 100755 tests/queries/0_stateless/03156_default_multiquery_split.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 3e613532f3a..1d99d223ee9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1135,8 +1135,6 @@ void Client::processOptions(const OptionsDescription & options_description, if ((query_fuzzer_runs = options["query-fuzzer-runs"].as())) { - // Fuzzer implies multiquery. - config().setBool("multiquery", true); // Ignore errors in parsing queries. config().setBool("ignore-error", true); ignore_error = true; @@ -1144,8 +1142,6 @@ void Client::processOptions(const OptionsDescription & options_description, if ((create_query_fuzzer_runs = options["create-query-fuzzer-runs"].as())) { - // Fuzzer implies multiquery. - config().setBool("multiquery", true); // Ignore errors in parsing queries. config().setBool("ignore-error", true); @@ -1201,9 +1197,6 @@ void Client::processConfig() } print_stack_trace = config().getBool("stacktrace", false); - if (config().has("multiquery")) - is_multiquery = true; - pager = config().getString("pager", ""); setDefaultFormatsAndCompressionFromConfiguration(); @@ -1359,13 +1352,6 @@ void Client::readArguments( allow_repeated_settings = true; else if (arg == "--allow_merge_tree_settings") allow_merge_tree_settings = true; - else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-')) - { - /// Transform the abbreviated syntax '--multiquery ' into the full syntax '--multiquery -q ' - ++arg_num; - arg = argv[arg_num]; - addMultiquery(arg, common_arguments); - } else if (arg == "--password" && ((arg_num + 1) >= argc || std::string_view(argv[arg_num + 1]).starts_with('-'))) { common_arguments.emplace_back(arg); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ce0e179939d..88d5a0253d1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -569,9 +569,6 @@ void LocalServer::processConfig() if (!queries.empty() && getClientConfiguration().has("queries-file")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time"); - if (getClientConfiguration().has("multiquery")) - is_multiquery = true; - pager = getClientConfiguration().getString("pager", ""); delayed_interactive = getClientConfiguration().has("interactive") && (!queries.empty() || getClientConfiguration().has("queries-file")); @@ -936,13 +933,6 @@ void LocalServer::readArguments(int argc, char ** argv, Arguments & common_argum query_parameters.emplace(param_continuation.substr(0, equal_pos), param_continuation.substr(equal_pos + 1)); } } - else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-')) - { - /// Transform the abbreviated syntax '--multiquery ' into the full syntax '--multiquery -q ' - ++arg_num; - arg = argv[arg_num]; - addMultiquery(arg, common_arguments); - } else { common_arguments.emplace_back(arg); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 78d8e7e3d6a..91cf737f292 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -80,6 +80,8 @@ #include #include "config.h" +#include +#include #if USE_GWP_ASAN # include @@ -745,14 +747,6 @@ void ClientBase::adjustSettings() /// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles) - /// In case of multi-query we allow data after semicolon since it will be - /// parsed by the client and interpreted as new query - if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) - { - settings.input_format_values_allow_data_after_semicolon = true; - settings.input_format_values_allow_data_after_semicolon.changed = false; - } - /// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty. if (!pager.empty() || !stdout_is_a_tty) { @@ -1525,13 +1519,6 @@ void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query) } -void ClientBase::addMultiquery(std::string_view query, Arguments & common_arguments) const -{ - common_arguments.emplace_back("--multiquery"); - common_arguments.emplace_back("-q"); - common_arguments.emplace_back(query); -} - namespace { bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) @@ -2186,23 +2173,48 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( return MultiQueryProcessingStage::PARSING_FAILED; } - // INSERT queries may have the inserted data in the query text - // that follow the query itself, e.g. "insert into t format CSV 1;2". - // They need special handling. First of all, here we find where the - // inserted data ends. In multi-query mode, it is delimited by a - // newline. - // The VALUES format needs even more handling - we also allow the - // data to be delimited by semicolon. This case is handled later by - // the format parser itself. - // We can't do multiline INSERTs with inline data, because most - // row input formats (e.g. TSV) can't tell when the input stops, - // unlike VALUES. + // INSERT queries may have the inserted data in the query text that follow the query itself, e.g. "insert into t format CSV 1,2". They + // need special handling. + // - If the INSERT statement FORMAT is VALUES, we use the VALUES format parser to skip the inserted data until we reach the trailing single semicolon. + // - Other formats (e.g. FORMAT CSV) are arbitrarily more complex and tricky to parse. For example, we may be unable to distinguish if the semicolon + // is part of the data or ends the statement. In this case, we simply assume that the end of the INSERT statement is determined by \n\n (two newlines). auto * insert_ast = parsed_query->as(); const char * query_to_execute_end = this_query_end; - if (insert_ast && insert_ast->data) { - this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); + if (insert_ast->format == "Values") + { + // Invoke the VALUES format parser to skip the inserted data + ReadBufferFromMemory data_in(insert_ast->data, all_queries_end - insert_ast->data); + skipBOMIfExists(data_in); + do + { + skipWhitespaceIfAny(data_in); + if (data_in.eof() || *data_in.position() == ';') + break; + } + while (ValuesBlockInputFormat::skipToNextRow(&data_in, 1, 0)); + // Handle the case of a comment followed by a semicolon + // Example: INSERT INTO tab VALUES xx; -- {serverError xx} + // If we use this error hint, the next query should not be placed on the same line + this_query_end = insert_ast->data + data_in.count(); + const auto * pos_newline = find_first_symbols<'\n'>(this_query_end, all_queries_end); + if (pos_newline != this_query_end) + { + TestHint hint(String(this_query_end, pos_newline - this_query_end)); + if (hint.hasClientErrors() || hint.hasServerErrors()) + this_query_end = pos_newline; + } + } + else + { + // Handling of generic formats + auto pos_newline = String(insert_ast->data, all_queries_end).find("\n\n"); + if (pos_newline != std::string::npos) + this_query_end = insert_ast->data + pos_newline; + else + this_query_end = all_queries_end; + } insert_ast->end = this_query_end; query_to_execute_end = isSyncInsertWithData(*insert_ast, client_context) ? insert_ast->data : this_query_end; } @@ -2237,7 +2249,10 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) size_t test_tags_length = getTestTagsLength(all_queries_text); /// Several queries separated by ';'. - /// INSERT data is ended by the end of line, not ';'. + /// INSERT data is ended by the empty line (\n\n), not ';'. + /// Unnecessary semicolons may cause data to be parsed containing ';' + /// e.g. 'insert into xx format csv val;' will insert "val;" instead of "val" + /// 'insert into xx format csv val\n;' will insert "val" and ";" /// An exception is VALUES format where we also support semicolon in /// addition to end of line. const char * this_query_begin = all_queries_text.data() + test_tags_length; @@ -2248,8 +2263,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) String query_to_execute; ASTPtr parsed_query; std::unique_ptr current_exception; - size_t retries_count = 0; + bool is_first = true; while (true) { @@ -2258,16 +2273,24 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) switch (stage) { case MultiQueryProcessingStage::QUERIES_END: + { + /// Compatible with old version when run interactive, e.g. "", "\ld" + if (is_first && is_interactive) + processTextAsSingleQuery(all_queries_text); + return true; + } case MultiQueryProcessingStage::PARSING_FAILED: { return true; } case MultiQueryProcessingStage::CONTINUE_PARSING: { + is_first = false; continue; } case MultiQueryProcessingStage::PARSING_EXCEPTION: { + is_first = false; this_query_end = find_first_symbols<'\n'>(this_query_end, all_queries_end); // Try to find test hint for syntax error. We don't know where @@ -2297,6 +2320,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) } case MultiQueryProcessingStage::EXECUTE_QUERY: { + is_first = false; full_query = all_queries_text.substr(this_query_begin - all_queries_text.data(), this_query_end - this_query_begin); if (query_fuzzer_runs) { @@ -2306,6 +2330,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) this_query_begin = this_query_end; continue; } + if (suggest) + updateSuggest(parsed_query); // Now we know for sure where the query ends. // Look for the hint in the text of query + insert data + trailing @@ -2456,14 +2482,6 @@ bool ClientBase::processQueryText(const String & text) return processMultiQueryFromFile(file_name); } - if (!is_multiquery) - { - assert(!query_fuzzer_runs); - processTextAsSingleQuery(text); - - return true; - } - if (query_fuzzer_runs) { processWithFuzzing(text); @@ -2901,9 +2919,9 @@ void ClientBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") - ("query,q", po::value>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))") + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") - ("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.") + ("multiquery,n", "Obsolete, does nothing") ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") ("database,d", po::value(), "database") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") @@ -2932,7 +2950,7 @@ void ClientBase::init(int argc, char ** argv) ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") - ("ignore-error", "do not stop processing in multiquery mode") + ("ignore-error", "do not stop processing when an error occurs") ("stacktrace", "print stack traces of exceptions") ("hardware-utilization", "print hardware utilization information in progress bar") ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") @@ -3025,8 +3043,6 @@ void ClientBase::init(int argc, char ** argv) queries_files = options["queries-file"].as>(); if (options.count("multiline")) getClientConfiguration().setBool("multiline", true); - if (options.count("multiquery")) - getClientConfiguration().setBool("multiquery", true); if (options.count("ignore-error")) getClientConfiguration().setBool("ignore-error", true); if (options.count("format")) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index e3d5fe8dc97..7689744a373 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -156,8 +156,6 @@ protected: void setInsertionTable(const ASTInsertQuery & insert_query); - void addMultiquery(std::string_view query, Arguments & common_arguments) const; - private: void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel); bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_); @@ -229,7 +227,6 @@ protected: std::unique_ptr signal_listener; bool is_interactive = false; /// Use either interactive line editing interface or batch mode. - bool is_multiquery = false; bool delayed_interactive = false; bool echo_queries = false; /// Print queries before execution in batch mode. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3349662cb78..4fc2034b855 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1156,7 +1156,6 @@ class IColumn; M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ - M(Bool, input_format_values_allow_data_after_semicolon, false, "For Values format: allow extra data after semicolon (used by client to interpret comments).", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ /** This setting is obsolete and do nothing, left for compatibility reasons. */ \ M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 501343d2ca9..efd467abb37 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -215,7 +215,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.tsv.allow_variable_number_of_columns = settings.input_format_tsv_allow_variable_number_of_columns; format_settings.tsv.crlf_end_of_line_input = settings.input_format_tsv_crlf_end_of_line; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; - format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c23d717d52f..157b61aa2af 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -625,8 +625,6 @@ void ValuesBlockInputFormat::readSuffix() skipWhitespaceIfAny(*buf); if (buf->hasUnreadData()) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon"); - if (!format_settings.values.allow_data_after_semicolon && !buf->eof()) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)"); return; } diff --git a/tests/queries/0_stateless/00366_multi_statements.reference b/tests/queries/0_stateless/00366_multi_statements.reference index 68f9261b937..71ca552522c 100644 --- a/tests/queries/0_stateless/00366_multi_statements.reference +++ b/tests/queries/0_stateless/00366_multi_statements.reference @@ -3,8 +3,10 @@ 1 1 Syntax error -Syntax error -Syntax error +1 +2 +1 +2 Syntax error Syntax error 1 diff --git a/tests/queries/0_stateless/00366_multi_statements.sh b/tests/queries/0_stateless/00366_multi_statements.sh index 9b885bb1b32..0b2e80fe457 100755 --- a/tests/queries/0_stateless/00366_multi_statements.sh +++ b/tests/queries/0_stateless/00366_multi_statements.sh @@ -10,8 +10,8 @@ $CLICKHOUSE_CLIENT --query="SELECT 1; " $CLICKHOUSE_CLIENT --query="SELECT 1 ; " $CLICKHOUSE_CLIENT --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_CLIENT -n --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' diff --git a/tests/queries/0_stateless/00499_json_enum_insert.sql b/tests/queries/0_stateless/00499_json_enum_insert.sql index c4988d08243..24eab38c65f 100644 --- a/tests/queries/0_stateless/00499_json_enum_insert.sql +++ b/tests/queries/0_stateless/00499_json_enum_insert.sql @@ -4,12 +4,15 @@ DROP TABLE IF EXISTS json; CREATE TABLE json (x Enum8('browser' = 1, 'mobile' = 2), y String) ENGINE = Memory; INSERT INTO json (y) VALUES ('Hello'); + SELECT * FROM json ORDER BY y; INSERT INTO json (y) FORMAT JSONEachRow {"y": "World 1"}; + SELECT * FROM json ORDER BY y; INSERT INTO json (x, y) FORMAT JSONEachRow {"y": "World 2"}; + SELECT * FROM json ORDER BY y; DROP TABLE json; diff --git a/tests/queries/0_stateless/00517_date_parsing.sql b/tests/queries/0_stateless/00517_date_parsing.sql index 9067d39ba3e..4cfede7cb2b 100644 --- a/tests/queries/0_stateless/00517_date_parsing.sql +++ b/tests/queries/0_stateless/00517_date_parsing.sql @@ -7,6 +7,7 @@ INSERT INTO date VALUES ('2017-01-02'), ('2017-1-02'), ('2017-01-2'), ('2017-1-2 SELECT * FROM date; INSERT INTO date FORMAT JSONEachRow {"d": "2017-01-02"}, {"d": "2017-1-02"}, {"d": "2017-01-2"}, {"d": "2017-1-2"}, {"d": "2017/01/02"}, {"d": "2017/1/02"}, {"d": "2017/01/2"}, {"d": "2017/1/2"}, {"d": "2017-11-12"}; + SELECT * FROM date ORDER BY d; DROP TABLE date; diff --git a/tests/queries/0_stateless/00560_float_leading_plus_in_exponent.sql b/tests/queries/0_stateless/00560_float_leading_plus_in_exponent.sql index 33e010cdc9c..d2d62510e7b 100644 --- a/tests/queries/0_stateless/00560_float_leading_plus_in_exponent.sql +++ b/tests/queries/0_stateless/00560_float_leading_plus_in_exponent.sql @@ -1,3 +1,5 @@ CREATE TEMPORARY TABLE test_float (x Float64); + INSERT INTO test_float FORMAT TabSeparated 1.075e+06 + SELECT * FROM test_float; diff --git a/tests/queries/0_stateless/00700_decimal_formats.sql b/tests/queries/0_stateless/00700_decimal_formats.sql index 45b2a5f1078..f008897bf5d 100644 --- a/tests/queries/0_stateless/00700_decimal_formats.sql +++ b/tests/queries/0_stateless/00700_decimal_formats.sql @@ -9,22 +9,22 @@ CREATE TABLE IF NOT EXISTS decimal INSERT INTO decimal (a, b, c) VALUES (42.0, -42.0, 42) (0.42, -0.42, .42) (42.42, -42.42, 42.42); INSERT INTO decimal (a, b, c) FORMAT JSONEachRow {"a":1.1, "b":-1.1, "c":1.1} {"a":1.0, "b":-1.0, "c":1} {"a":0.1, "b":-0.1, "c":.1}; + INSERT INTO decimal (a, b, c) FORMAT CSV 2.0,-2.0,2 -; + INSERT INTO decimal (a, b, c) FORMAT CSV 0.2 ,-0.2 ,.2 -; + INSERT INTO decimal (a, b, c) FORMAT CSV 2.2 , -2.2 , 2.2 -; + INSERT INTO decimal (a, b, c) FORMAT TabSeparated 3.3 -3.3 3.3 -; + INSERT INTO decimal (a, b, c) FORMAT TabSeparated 3.0 -3.0 3 -; + INSERT INTO decimal (a, b, c) FORMAT TabSeparated 0.3 -0.3 .3 -; + INSERT INTO decimal (a, b, c) FORMAT CSV 4.4E+5,-4E+8,.4E+20 -; + INSERT INTO decimal (a, b, c) FORMAT CSV 5.5e-2, -5e-9 ,.5e-17 -; SELECT * FROM decimal ORDER BY a FORMAT JSONEachRow; SELECT * FROM decimal ORDER BY b DESC FORMAT CSV; diff --git a/tests/queries/0_stateless/00760_insert_json_with_defaults.sql b/tests/queries/0_stateless/00760_insert_json_with_defaults.sql index 1430d10cdb0..54987258375 100644 --- a/tests/queries/0_stateless/00760_insert_json_with_defaults.sql +++ b/tests/queries/0_stateless/00760_insert_json_with_defaults.sql @@ -15,20 +15,29 @@ CREATE TABLE defaults ) ENGINE = MergeTree ORDER BY x; INSERT INTO defaults FORMAT JSONEachRow {"x":1, "y":1}; + INSERT INTO defaults (x, y) SELECT x, y FROM defaults LIMIT 1; + INSERT INTO defaults FORMAT JSONEachRow {"x":2, "y":2, "c":2}; + INSERT INTO defaults FORMAT JSONEachRow {"x":3, "y":3, "a":3, "b":3, "c":3}; + INSERT INTO defaults FORMAT JSONEachRow {"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6}; SELECT * FROM defaults ORDER BY (x, y); ALTER TABLE defaults ADD COLUMN n Nested(a UInt64, b String); + INSERT INTO defaults FORMAT JSONEachRow {"x":7, "y":7, "n.a":[1,2], "n.b":["a","b"]}; + SELECT * FROM defaults WHERE x = 7 FORMAT JSONEachRow; ALTER TABLE defaults ADD COLUMN n.c Array(UInt8) DEFAULT arrayMap(x -> 0, n.a) AFTER n.a; + INSERT INTO defaults FORMAT JSONEachRow {"x":8, "y":8, "n.a":[3,4], "n.b":["c","d"]}; + INSERT INTO defaults FORMAT JSONEachRow {"x":9, "y":9}; + SELECT * FROM defaults WHERE x > 7 ORDER BY x FORMAT JSONEachRow; DROP TABLE defaults; diff --git a/tests/queries/0_stateless/00861_decimal_quoted_csv.sql b/tests/queries/0_stateless/00861_decimal_quoted_csv.sql index e716c607ef8..d7211c55df7 100644 --- a/tests/queries/0_stateless/00861_decimal_quoted_csv.sql +++ b/tests/queries/0_stateless/00861_decimal_quoted_csv.sql @@ -2,15 +2,14 @@ DROP TABLE IF EXISTS test_00861; CREATE TABLE test_00861 (key UInt64, d32 Decimal32(2), d64 Decimal64(2), d128 Decimal128(2)) ENGINE = Memory; INSERT INTO test_00861 FORMAT CSV "1","1","1","1" -; + INSERT INTO test_00861 FORMAT CSV "2","-1","-1","-1" -; + INSERT INTO test_00861 FORMAT CSV "3","1.0","1.0","1.0" -; + INSERT INTO test_00861 FORMAT CSV "4","-0.1","-0.1","-0.1" -; + INSERT INTO test_00861 FORMAT CSV "5","0.010","0.010","0.010" -; SELECT * FROM test_00861 ORDER BY key; diff --git a/tests/queries/0_stateless/00945_ml_test.sql b/tests/queries/0_stateless/00945_ml_test.sql index 299b550bceb..c3a3a51b98c 100644 --- a/tests/queries/0_stateless/00945_ml_test.sql +++ b/tests/queries/0_stateless/00945_ml_test.sql @@ -7,7 +7,7 @@ CREATE TABLE IF NOT EXISTS defaults predict1 Float64, predict2 Float64 ) ENGINE = Memory; -insert into defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2) +insert into defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2); DROP TABLE IF EXISTS model; create table model engine = Memory as select stochasticLogisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from defaults; diff --git a/tests/queries/0_stateless/00946_ml_test.sql b/tests/queries/0_stateless/00946_ml_test.sql index b3f4941c2a7..a3da3b7e5c7 100644 --- a/tests/queries/0_stateless/00946_ml_test.sql +++ b/tests/queries/0_stateless/00946_ml_test.sql @@ -18,7 +18,7 @@ CREATE TABLE IF NOT EXISTS defaults predict7 Float64 ) ENGINE = Memory; -insert into defaults values (1.76210664421617,1.7469706406568504,0.7988286239230257,1.0938642223599824,1.167321139201246,1.7648182796261376,0.909111664354187,0.92,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,-0.06763531281107672,0.7988286239230257,0.5966532121963541,1.167321139201246,0.4551512643242912,0.909111664354187,0.76,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.07046681268810527,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-1.0056311758200724,0.909111664354187,0.72,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,0.11933920911869125,0.909111664354187,0.8,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.24499761810756004,-0.7274920231630502,-0.952028633990455,-1.3921908284581592,-0.5042604443804907,-0.6530285178541898,-1.0999748867047898,0.65,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.1512488252480781,1.2520781078928702,1.674257252879766,1.0938642223599824,-0.5042604443804907,1.244309594057455,0.909111664354187,0.9,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,0.6101272780073337,-0.6698191206144725,0.909111664354187,0.75,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.7685900343659244,-1.057420378339037,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-1.1735372034228724,-1.0999748867047898,0.68,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-1.2921824506242887,-0.8924562007510436,-1.8274572629471952,-1.3921908284581592,-2.175842027962227,-1.0056311758200724,-1.0999748867047898,0.5,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.5403910062799865,0.09732886477691666,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.0018049897967303734,-1.0999748867047898,0.45,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.7149218116994412,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.3340070654088696,0.909111664354187,0.52,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.889452617118896,0.5922213975408968,0.7988286239230257,0.5966532121963541,1.167321139201246,0.6734291002079332,0.909111664354187,0.84,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.9767180198286235,0.7571855751288902,0.7988286239230257,0.5966532121963541,1.167321139201246,0.8413351278107333,0.909111664354187,0.78,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.8558554370756518,0.26229304236491,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.62,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,-1.618648166768315,-0.6698191206144725,0.909111664354187,0.61,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.24499761810756004,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.5019130930116695,-1.0999748867047898,0.54,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.016798590021622126,-0.06763531281107672,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,0.16971101739953035,-1.0999748867047898,0.66,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.1913293954410769,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.65,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.10406399273134952,0.4272572199529034,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,0.3376170450023333,-1.0999748867047898,0.63,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-1.2049170479145614,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.1661010378060696,-1.0999748867047898,0.62,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,-1.618648166768315,-1.1735372034228724,0.909111664354187,0.64,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.7149218116994412,1.087113930304877,0.7988286239230257,-0.3977688081309026,-1.618648166768315,-0.3340070654088696,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.9767180198286235,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.5129592382219361,0.909111664354187,0.94,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(1.5003104360869879,1.9119348182448437,1.674257252879766,1.5910752325236108,1.167321139201246,1.848771293427536,0.909111664354187,0.95,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(1.6748412415064426,1.9119348182448437,1.674257252879766,0.5966532121963541,0.052933416813421515,2.016677321030339,0.909111664354187,0.97,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(2.023902852345352,2.076898995832837,1.674257252879766,1.0938642223599824,1.167321139201246,1.680865265824736,0.909111664354187,0.94,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.4531256035702591,0.26229304236491,1.674257252879766,1.0938642223599824,0.052933416813421515,0.3376170450023333,-1.0999748867047898,0.76,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.6412440614631982,-1.5523129111030172,-0.952028633990455,-1.8894018386217877,-1.0614543055744028,-1.8451613138340752,0.909111664354187,0.44,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.9030402695923805,-2.377133799042984,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-2.348879396642477,-1.0999748867047898,0.46,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-0.5940592289464697,-1.3873487335150236,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-2.180973369039677,-1.0999748867047898,0.54,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.4667132560437435,-1.7172770886910105,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.8377251482172725,0.909111664354187,0.65,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.889452617118896,-0.7274920231630502,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.5019130930116695,0.909111664354187,0.74,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(1.8493720469258974,1.7469706406568504,0.7988286239230257,-0.3977688081309026,1.167321139201246,1.3450532106191362,0.909111664354187,0.91,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(2.023902852345352,1.087113930304877,1.674257252879766,0.5966532121963541,0.6101272780073337,1.680865265824736,0.909111664354187,0.9,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(1.2385142279578056,0.7571855751288902,1.674257252879766,0.5966532121963541,1.724515000395158,2.016677321030339,0.909111664354187,0.94,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.2785947981508043,0.4272572199529034,1.674257252879766,1.5910752325236108,1.724515000395158,1.0092411554135332,0.909111664354187,0.88,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.553978658753471,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.6101272780073337,-0.3340070654088696,-1.0999748867047898,0.64,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.4667132560437435,-0.39756366798706344,-1.8274572629471952,-2.386612848785416,-1.618648166768315,-1.3414432310256739,-1.0999748867047898,0.58,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-1.117651645204834,-0.39756366798706344,-1.8274572629471952,-0.3977688081309026,-2.175842027962227,-1.8451613138340752,-1.0999748867047898,0.52,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.8558554370756518,0.09732886477691666,-0.952028633990455,0.5966532121963541,0.052933416813421515,-1.5093492586284738,-1.0999748867047898,0.48,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.7685900343659244,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.46,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.07046681268810527,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6698191206144725,0.909111664354187,0.49,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.3322630208172874,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.1661010378060696,0.909111664354187,0.53,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.3257796306675331,1.582006463068857,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.8413351278107333,-1.0999748867047898,0.87,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.8021872144091686,0.9221497527168835,1.674257252879766,1.0938642223599824,0.6101272780073337,1.3450532106191362,0.909111664354187,0.91,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.4531256035702591,0.4272572199529034,1.674257252879766,1.5910752325236108,0.6101272780073337,0.8413351278107333,0.909111664354187,0.88,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.0639834225383509,1.087113930304877,1.674257252879766,0.5966532121963541,1.724515000395158,1.1771471830163363,0.909111664354187,0.86,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.9366374496356247,1.9119348182448437,1.674257252879766,1.0938642223599824,0.6101272780073337,1.848771293427536,-1.0999748867047898,0.89,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.36586020086053167,0.4272572199529034,-0.0766000050337147,0.09944220203272576,1.724515000395158,0.42157005880373183,0.909111664354187,0.82,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.889452617118896,0.5922213975408968,0.7988286239230257,-0.3977688081309026,0.6101272780073337,-0.3340070654088696,0.909111664354187,0.78,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.3322630208172874,-1.5523129111030172,-0.0766000050337147,-0.8949798182945309,1.167321139201246,-0.5019130930116695,0.909111664354187,0.76,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.41952842352701486,-1.2223845559270303,-0.952028633990455,-1.8894018386217877,0.052933416813421515,-1.1735372034228724,0.909111664354187,0.56,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.5003104360869879,1.4170422854808635,0.7988286239230257,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.78,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.6276564089897139,0.7571855751288902,0.7988286239230257,0.5966532121963541,-1.0614543055744028,-0.8377251482172725,0.909111664354187,0.72,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-1.0056311758200724,-1.0999748867047898,0.7,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.2785947981508043,-0.7274920231630502,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-1.5093492586284738,-1.0999748867047898,0.64,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.07046681268810527,-0.8924562007510436,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-2.013067341436875,-1.0999748867047898,0.64,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.6412440614631982,-1.3873487335150236,-0.952028633990455,0.5966532121963541,-1.618648166768315,-1.6772552862312753,-1.0999748867047898,0.46,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.4667132560437435,-1.3873487335150236,-1.8274572629471952,-0.3977688081309026,-1.618648166768315,-3.0205035070536796,0.909111664354187,0.36,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.5067938262367422,-0.5625278455750569,-0.952028633990455,-1.3921908284581592,-1.618648166768315,-0.5019130930116695,-1.0999748867047898,0.42,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.681324631656197,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.8377251482172725,-1.0999748867047898,0.48,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.8558554370756518,-1.057420378339037,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-0.6698191206144725,-1.0999748867047898,0.47,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.117651645204834,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.6698191206144725,0.909111664354187,0.54,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.15773221539783266,-0.06763531281107672,-0.952028633990455,0.5966532121963541,-0.5042604443804907,-0.1661010378060696,0.909111664354187,0.56,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.7149218116994412,0.5922213975408968,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.16971101739953035,-1.0999748867047898,0.52,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.09944220203272576,0.052933416813421515,0.5391042781256927,-1.0999748867047898,0.55,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.889452617118896,1.087113930304877,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.7070103057284927,-1.0999748867047898,0.61,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.07046681268810527,-0.06763531281107672,-0.952028633990455,0.09944220203272576,0.052933416813421515,0.06896740083785215,0.909111664354187,0.57,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.10406399273134952,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.6101272780073337,1.0428223609340956,0.909111664354187,0.68,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.9767180198286235,1.2520781078928702,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9420787443724145,0.909111664354187,0.78,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(1.3257796306675331,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.7480276768658578,0.909111664354187,0.94,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(1.6748412415064426,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,1.9495149099892173,0.909111664354187,0.96,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.36586020086053167,0.5922213975408968,1.674257252879766,1.5910752325236108,1.724515000395158,1.4290062244205346,0.909111664354187,0.93,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.24499761810756004,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.7405915112490521,0.909111664354187,0.84,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.24499761810756004,-0.2325994903990701,-0.0766000050337147,-0.3977688081309026,1.724515000395158,0.5055230726051333,-1.0999748867047898,0.74,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(1.0639834225383509,1.087113930304877,-0.952028633990455,-1.3921908284581592,0.6101272780073337,-0.06535742124438843,0.909111664354187,0.72,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.889452617118896,0.7571855751288902,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.20329222292009272,0.909111664354187,0.74,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.3794478533340162,-1.3873487335150236,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.6362379150939101,-1.0999748867047898,0.64,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.8157748668826532,-2.0472054438669973,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-1.777998902792955,0.909111664354187,0.44,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.990305672302108,-2.377133799042984,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-2.0802297524779956,-1.0999748867047898,0.46,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-0.972049970299513,0.909111664354187,0.5,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(2.023902852345352,2.076898995832837,0.7988286239230257,1.5910752325236108,1.724515000395158,1.5129592382219361,0.909111664354187,0.96,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.2785947981508043,0.4272572199529034,1.674257252879766,1.5910752325236108,1.167321139201246,1.0428223609340956,0.909111664354187,0.92,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.4531256035702591,1.2520781078928702,1.674257252879766,0.5966532121963541,1.167321139201246,1.2778907995780144,0.909111664354187,0.92,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(2.023902852345352,1.2520781078928702,1.674257252879766,1.0938642223599824,1.167321139201246,1.4290062244205346,0.909111664354187,0.94,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.1913293954410769,-0.7274920231630502,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.10254860635841155,-1.0999748867047898,0.76,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-0.15773221539783266,-0.2325994903990701,-0.0766000050337147,1.0938642223599824,0.052933416813421515,-0.30042585988831016,-1.0999748867047898,0.72,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.016798590021622126,-0.06763531281107672,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.535494298532232,-1.0999748867047898,0.66,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.24499761810756004,0.09732886477691666,-0.0766000050337147,1.0938642223599824,0.052933416813421515,-0.7705627371761508,-1.0999748867047898,0.64,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.07046681268810527,0.26229304236491,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.27045463396121155,0.909111664354187,0.74,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(0.10406399273134952,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.6101272780073337,-1.139955997902313,0.909111664354187,0.64,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.553978658753471,-1.7172770886910105,-0.0766000050337147,1.5910752325236108,0.052933416813421515,-1.5765116696695942,-1.0999748867047898,0.38,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.6412440614631982,-1.5523129111030172,-0.952028633990455,0.5966532121963541,-0.5042604443804907,-0.9552593675392334,-1.0999748867047898,0.34,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.3794478533340162,-1.7172770886910105,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-1.2071184089434333,0.909111664354187,0.44,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.2049170479145614,-1.3873487335150236,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898,0.36,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.117651645204834,-1.2223845559270303,0.7988286239230257,-1.8894018386217877,-1.0614543055744028,-1.2742808199845537,-1.0999748867047898,0.42,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.9431208397853792,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-1.0056311758200724,-1.0999748867047898,0.48,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(1.2385142279578056,2.076898995832837,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.6062666891668145,0.909111664354187,0.86,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(1.3257796306675331,1.9119348182448437,0.7988286239230257,1.5910752325236108,1.167321139201246,1.076403566454655,0.909111664354187,0.9,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(0.5403910062799865,0.9221497527168835,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.47194186708457386,0.909111664354187,0.79,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.4531256035702591,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.2332634488471884,0.909111664354187,0.71,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.8041439426967131,-1.0999748867047898,0.64,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.24499761810756004,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.052933416813421515,-0.585866106813071,-1.0999748867047898,0.62,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.016798590021622126,-0.5625278455750569,-0.952028633990455,1.0938642223599824,0.6101272780073337,-0.2164728460869087,-1.0999748867047898,0.57,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.8021872144091686,0.7571855751288902,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.7573821140093348,0.909111664354187,0.74,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.07046681268810527,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.30403583948177093,0.909111664354187,0.69,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,1.167321139201246,0.9756599498929738,0.909111664354187,0.87,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.8493720469258974,1.582006463068857,0.7988286239230257,0.09944220203272576,1.167321139201246,1.4457968271808173,0.909111664354187,0.91,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.2385142279578056,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.3114720050985766,0.909111664354187,0.93,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.117651645204834,-0.7274920231630502,1.674257252879766,1.5910752325236108,0.6101272780073337,0.06896740083785215,-1.0999748867047898,0.68,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.0303862424951067,0.09732886477691666,1.674257252879766,-0.3977688081309026,-0.5042604443804907,-0.199682243326629,-1.0999748867047898,0.61,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.36586020086053167,0.26229304236491,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.13612981187897094,0.909111664354187,0.69,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.3794478533340162,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.4347506819705508,0.909111664354187,0.62,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(0.2785947981508043,0.4272572199529034,-0.952028633990455,0.5966532121963541,0.052933416813421515,-0.06535742124438843,-1.0999748867047898,0.72,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5067938262367422,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.2500540516074711,0.909111664354187,0.59,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,0.7988286239230257,1.0938642223599824,1.167321139201246,0.7405915112490521,0.909111664354187,0.66,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.553978658753471,-0.8924562007510436,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.03538619531728977,-1.0999748867047898,0.56,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-2.3393672831410175,-0.5625278455750569,0.7988286239230257,-1.3921908284581592,-1.0614543055744028,-1.9123237248751956,-1.0999748867047898,0.45,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.8157748668826532,-1.3873487335150236,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-2.214554574560236,-1.0999748867047898,0.47,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(0.889452617118896,-0.5625278455750569,1.674257252879766,-0.3977688081309026,0.052933416813421515,0.4047794560434521,0.909111664354187,0.71,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,1.6137028547836172,0.909111664354187,0.94,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(1.5003104360869879,1.9119348182448437,1.674257252879766,1.0938642223599824,1.167321139201246,1.4793780327013768,0.909111664354187,0.94,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,0.7988286239230257,-1.8894018386217877,-1.0614543055744028,-0.40116947644999135,-1.0999748867047898,0.57,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.7685900343659244,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.6362379150939101,-1.0999748867047898,0.61,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.3794478533340162,-0.2325994903990701,0.7988286239230257,-0.8949798182945309,-0.5042604443804907,-0.2164728460869087,-1.0999748867047898,0.57,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.10254860635841155,0.909111664354187,0.64,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.5403910062799865,0.9221497527168835,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,1.2107283885368956,0.909111664354187,0.85,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.1913293954410769,0.7571855751288902,-0.0766000050337147,-0.8949798182945309,-1.618648166768315,0.18650162015981303,0.909111664354187,0.78,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.8021872144091686,0.7571855751288902,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.8413351278107333,0.909111664354187,0.84,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.4130450333772604,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.2611001968177347,0.909111664354187,0.92,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.9366374496356247,1.087113930304877,1.674257252879766,0.5966532121963541,1.167321139201246,1.9495149099892173,0.909111664354187,0.96,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-1.2049170479145614,-0.39756366798706344,1.674257252879766,1.5910752325236108,1.167321139201246,0.08575800359813185,-1.0999748867047898,0.77,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.681324631656197,-0.39756366798706344,1.674257252879766,0.09944220203272576,0.052933416813421515,-0.06535742124438843,-1.0999748867047898,0.71,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.5403910062799865,0.7571855751288902,1.674257252879766,0.5966532121963541,1.167321139201246,0.30403583948177093,-1.0999748867047898,0.79,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.4130450333772604,0.9221497527168835,1.674257252879766,0.5966532121963541,0.6101272780073337,1.1435659774957738,0.909111664354187,0.89,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.24499761810756004,0.26229304236491,0.7988286239230257,0.09944220203272576,0.6101272780073337,0.2872452367214912,0.909111664354187,0.82,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,1.5910752325236108,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.76,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(-0.07046681268810527,-1.2223845559270303,-0.952028633990455,-1.8894018386217877,-0.5042604443804907,-0.7369815316555913,0.909111664354187,0.71,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.8021872144091686,1.4170422854808635,-0.952028633990455,1.0938642223599824,-0.5042604443804907,0.8077539222901738,0.909111664354187,0.8,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.10406399273134952,0.26229304236491,-1.8274572629471952,0.09944220203272576,0.052933416813421515,0.8749163333312926,-1.0999748867047898,0.78,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.0639834225383509,0.4272572199529034,-0.952028633990455,0.5966532121963541,-0.5042604443804907,0.9252881416121347,0.909111664354187,0.84,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.3257796306675331,1.7469706406568504,-0.952028633990455,1.0938642223599824,0.052933416813421515,1.2778907995780144,0.909111664354187,0.9,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.2385142279578056,1.2520781078928702,1.674257252879766,0.5966532121963541,0.052933416813421515,1.412215621660255,0.909111664354187,0.92,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(2.023902852345352,2.076898995832837,0.7988286239230257,1.0938642223599824,0.6101272780073337,2.2181645541536983,0.909111664354187,0.97,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.7149218116994412,0.7571855751288902,-0.952028633990455,-0.3977688081309026,0.052933416813421515,0.6062666891668145,0.909111664354187,0.8,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.2785947981508043,0.9221497527168835,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,0.06896740083785215,0.909111664354187,0.81,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(-0.15773221539783266,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-0.199682243326629,-1.0999748867047898,0.75,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.8021872144091686,1.087113930304877,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.8581257305710129,0.909111664354187,0.83,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.9366374496356247,1.4170422854808635,0.7988286239230257,0.5966532121963541,0.052933416813421515,2.016677321030339,0.909111664354187,0.96,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.5067938262367422,-0.2325994903990701,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.5690755040527913,0.909111664354187,0.79,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(1.5003104360869879,1.087113930304877,0.7988286239230257,0.5966532121963541,0.6101272780073337,1.3954250188999753,0.909111664354187,0.93,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(1.3257796306675331,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.1435659774957738,0.909111664354187,0.94,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.36586020086053167,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,0.7741727167696144,0.909111664354187,0.86,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.6276564089897139,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.6101272780073337,0.25366403120093184,-1.0999748867047898,0.79,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.8021872144091686,0.09732886477691666,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.4887324698448536,-1.0999748867047898,0.8,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.41952842352701486,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.15292041463925066,-1.0999748867047898,0.77,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.15773221539783266,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-0.4347506819705508,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.681324631656197,-0.5625278455750569,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.5690755040527913,-1.0999748867047898,0.65,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.9431208397853792,-0.2325994903990701,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.7705627371761508,-1.0999748867047898,0.61,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-1.7285094641729257,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.1735372034228724,-1.0999748867047898,0.52,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.15773221539783266,-0.7274920231630502,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.57,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-1.6412440614631982,-1.3873487335150236,-1.8274572629471952,-1.8894018386217877,-0.5042604443804907,-1.9123237248751956,-1.0999748867047898,0.53,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.10406399273134952,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.1661010378060696,-1.0999748867047898,0.67,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.016798590021622126,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.06535742124438843,-1.0999748867047898,0.68,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.690219702968213,0.909111664354187,0.81,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,1.674257252879766,1.0938642223599824,0.6101272780073337,0.6230572919270941,-1.0999748867047898,0.78,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-1.2921824506242887,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,1.724515000395158,-0.45154128473083044,-1.0999748867047898,0.65,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-0.5522849012925116,-1.0999748867047898,0.64,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-2.0775710750118352,-1.7172770886910105,-0.952028633990455,-1.3921908284581592,0.6101272780073337,-1.3414432310256739,0.909111664354187,0.64,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.5067938262367422,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.0392123813406318,-1.0999748867047898,0.65,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.41952842352701486,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.9384687647789537,0.909111664354187,0.68,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(1.5003104360869879,1.582006463068857,1.674257252879766,0.5966532121963541,1.167321139201246,0.7909633195298942,0.909111664354187,0.89,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,0.7988286239230257,0.5966532121963541,1.724515000395158,0.8917069360915754,0.909111664354187,0.86,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.5403910062799865,0.9221497527168835,0.7988286239230257,0.5966532121963541,1.167321139201246,1.0596129636943752,0.909111664354187,0.89,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.36586020086053167,0.5922213975408968,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.6230572919270941,0.909111664354187,0.87,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.2785947981508043,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.4551512643242912,0.909111664354187,0.85,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(1.0639834225383509,1.9119348182448437,0.7988286239230257,1.0938642223599824,1.167321139201246,0.9420787443724145,0.909111664354187,0.9,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.1913293954410769,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,-1.0999748867047898,0.82,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.681324631656197,0.09732886477691666,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.8041439426967131,-1.0999748867047898,0.72,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.8558554370756518,-0.8924562007510436,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.5522849012925116,-1.0999748867047898,0.73,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.4667132560437435,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.7369815316555913,-1.0999748867047898,0.71,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.0303862424951067,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.30042585988831016,-1.0999748867047898,0.71,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-1.2071184089434333,-1.0999748867047898,0.68,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.24499761810756004,0.4272572199529034,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.3376170450023333,-1.0999748867047898,0.75,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.07046681268810527,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.46833188749111015,-1.0999748867047898,0.72,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.889452617118896,0.9221497527168835,0.7988286239230257,1.0938642223599824,1.167321139201246,0.8581257305710129,0.909111664354187,0.89,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.016798590021622126,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.13612981187897094,0.909111664354187,0.84,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.5875758387967152,1.7469706406568504,1.674257252879766,1.0938642223599824,0.052933416813421515,1.412215621660255,0.909111664354187,0.93,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.2385142279578056,1.2520781078928702,1.674257252879766,1.0938642223599824,0.052933416813421515,1.2778907995780144,0.909111664354187,0.93,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.6276564089897139,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,0.8077539222901738,0.909111664354187,0.88,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.6276564089897139,0.5922213975408968,1.674257252879766,1.0938642223599824,0.6101272780073337,0.9420787443724145,0.909111664354187,0.9,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.5403910062799865,0.4272572199529034,1.674257252879766,0.5966532121963541,1.724515000395158,0.6398478946873739,0.909111664354187,0.87,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.4531256035702591,1.087113930304877,1.674257252879766,1.0938642223599824,0.6101272780073337,0.572685483646252,0.909111664354187,0.86,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.6748412415064426,1.7469706406568504,1.674257252879766,1.0938642223599824,1.724515000395158,1.5633310465027752,0.909111664354187,0.94,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.07046681268810527,0.26229304236491,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.27045463396121155,-1.0999748867047898,0.77,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.8558554370756518,-0.06763531281107672,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.1325198322855102,0.909111664354187,0.78,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.9431208397853792,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,-0.5690755040527913,-1.0999748867047898,0.73,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,-0.952028633990455,0.09944220203272576,-1.0614543055744028,-0.45154128473083044,-1.0999748867047898,0.73,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.0766000050337147,1.0938642223599824,1.167321139201246,-0.2836352571280305,-1.0999748867047898,0.7,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.3322630208172874,-0.06763531281107672,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.15292041463925066,-1.0999748867047898,0.72,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(0.016798590021622126,-0.7274920231630502,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.0989386267649508,0.909111664354187,0.73,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.15773221539783266,0.4272572199529034,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.2332634488471884,0.909111664354187,0.72,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(2.023902852345352,2.076898995832837,1.674257252879766,1.0938642223599824,1.167321139201246,2.2013739513934185,0.909111664354187,0.97,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(1.5003104360869879,2.076898995832837,1.674257252879766,0.5966532121963541,1.724515000395158,2.1342115403522968,0.909111664354187,0.97,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.6412440614631982,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.0989386267649508,-1.0999748867047898,0.69,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.9030402695923805,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5933022724298738,-1.0999748867047898,0.57,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.15773221539783266,-1.3873487335150236,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-1.1903278061831537,-1.0999748867047898,0.63,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.5940592289464697,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.972049970299513,0.909111664354187,0.66,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.0303862424951067,-0.2325994903990701,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.7369815316555913,-1.0999748867047898,0.64,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.3794478533340162,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.8041439426967131,0.909111664354187,0.68,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(0.7149218116994412,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.7741727167696144,0.909111664354187,0.79,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.9767180198286235,0.4272572199529034,0.7988286239230257,1.5910752325236108,0.6101272780073337,0.908497538851855,0.909111664354187,0.82,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.8493720469258974,2.076898995832837,0.7988286239230257,1.5910752325236108,1.724515000395158,1.7816088823864173,0.909111664354187,0.95,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.4130450333772604,1.9119348182448437,1.674257252879766,1.5910752325236108,1.167321139201246,1.9830961155097766,0.909111664354187,0.96,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.2385142279578056,1.582006463068857,0.7988286239230257,1.0938642223599824,1.724515000395158,1.3786344161396955,0.909111664354187,0.94,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.1512488252480781,1.4170422854808635,1.674257252879766,1.5910752325236108,1.167321139201246,1.2778907995780144,0.909111664354187,0.93,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.4531256035702591,0.7571855751288902,0.7988286239230257,1.0938642223599824,1.167321139201246,1.1099847719752143,0.909111664354187,0.91,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.36586020086053167,0.26229304236491,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.8917069360915754,0.909111664354187,0.85,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.6276564089897139,0.4272572199529034,0.7988286239230257,-0.3977688081309026,0.052933416813421515,0.6230572919270941,0.909111664354187,0.84,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.41952842352701486,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.30042585988831016,-1.0999748867047898,0.74,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.3322630208172874,-0.7274920231630502,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.25366403120093184,-1.0999748867047898,0.76,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.07046681268810527,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.75,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.6276564089897139,0.9221497527168835,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.32082644224205065,-1.0999748867047898,0.76,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.7685900343659244,0.26229304236491,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.71,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.0303862424951067,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.6194473123336305,-1.0999748867047898,0.67,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.8157748668826532,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.9552593675392334,-1.0999748867047898,0.61,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.9431208397853792,0.4272572199529034,-0.952028633990455,0.09944220203272576,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.63,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.41952842352701486,0.4272572199529034,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.1157292295252305,-1.0999748867047898,0.64,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.10406399273134952,0.7571855751288902,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.11933920911869125,-1.0999748867047898,0.71,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.6276564089897139,0.5922213975408968,0.7988286239230257,-0.3977688081309026,-0.5042604443804907,0.690219702968213,0.909111664354187,0.82,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.3322630208172874,-0.5625278455750569,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.08575800359813185,-1.0999748867047898,0.73,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.1913293954410769,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.45154128473083044,0.909111664354187,0.74,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.41952842352701486,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.5522849012925116,-1.0999748867047898,0.69,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.117651645204834,-1.2223845559270303,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.8880969564981116,-1.0999748867047898,0.64,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(1.1512488252480781,0.9221497527168835,1.674257252879766,1.5910752325236108,0.6101272780073337,1.1939377857766158,0.909111664354187,0.91,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.8021872144091686,0.5922213975408968,1.674257252879766,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187,0.88,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.5966532121963541,1.167321139201246,0.9588693471326941,0.909111664354187,0.85,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(1.0639834225383509,1.087113930304877,1.674257252879766,1.0938642223599824,1.724515000395158,0.9924505526532535,0.909111664354187,0.86,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.5940592289464697,-0.5625278455750569,-0.0766000050337147,-1.3921908284581592,0.052933416813421515,-0.3843788736897117,-1.0999748867047898,0.7,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.1903278061831537,-1.0999748867047898,0.59,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-1.8157748668826532,-1.057420378339037,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-1.5429304641490347,-1.0999748867047898,0.6,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.016798590021622126,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.753772134415871,-1.0999748867047898,0.65,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.6276564089897139,1.2520781078928702,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.27045463396121155,0.909111664354187,0.7,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.7149218116994412,1.087113930304877,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.7405915112490521,0.909111664354187,0.76,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.24499761810756004,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.63,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.9767180198286235,0.4272572199529034,0.7988286239230257,0.5966532121963541,-1.0614543055744028,0.7070103057284927,0.909111664354187,0.81,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.07046681268810527,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.22008282568037243,-1.0999748867047898,0.72,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.199682243326629,-1.0999748867047898,0.71,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4551512643242912,0.909111664354187,0.8,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,0.5922213975408968,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.3879888532831724,0.909111664354187,0.77,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,-0.04856681848410872,0.909111664354187,0.74,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.07046681268810527,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.6734291002079332,-1.0999748867047898,0.7,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.10406399273134952,-1.2223845559270303,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.0989386267649508,0.909111664354187,0.71,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.5875758387967152,1.2520781078928702,0.7988286239230257,1.0938642223599824,1.167321139201246,1.8151900879069767,0.909111664354187,0.93,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,1.087113930304877,0.7988286239230257,0.5966532121963541,1.724515000395158,0.8749163333312926,-1.0999748867047898,0.85,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.8558554370756518,0.4272572199529034,0.7988286239230257,0.5966532121963541,1.167321139201246,-0.3843788736897117,-1.0999748867047898,0.79,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.681324631656197,-1.3873487335150236,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.76,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,-1.2223845559270303,-0.0766000050337147,0.5966532121963541,1.724515000395158,0.06896740083785215,0.909111664354187,0.78,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.8021872144091686,-0.8924562007510436,0.7988286239230257,1.5910752325236108,1.724515000395158,0.27045463396121155,0.909111664354187,0.77,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.2385142279578056,1.9119348182448437,0.7988286239230257,1.5910752325236108,1.167321139201246,1.244309594057455,0.909111664354187,0.9,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.889452617118896,0.09732886477691666,1.674257252879766,1.5910752325236108,0.052933416813421515,0.8917069360915754,0.909111664354187,0.87,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.41952842352701486,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.8545157509775522,-1.0999748867047898,0.71,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.7685900343659244,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.40116947644999135,0.909111664354187,0.7,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.6276564089897139,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-2.175842027962227,0.32082644224205065,0.909111664354187,0.7,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.7149218116994412,0.4272572199529034,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,0.27045463396121155,0.909111664354187,0.75,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,-0.8949798182945309,-1.0614543055744028,0.13612981187897094,-1.0999748867047898,0.71,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.2500540516074711,-1.0999748867047898,0.72,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.24499761810756004,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.7201909288953117,0.909111664354187,0.73,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.889452617118896,0.9221497527168835,0.7988286239230257,1.0938642223599824,1.724515000395158,0.908497538851855,-1.0999748867047898,0.83,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.7685900343659244,0.09732886477691666,0.7988286239230257,1.0938642223599824,1.724515000395158,-0.4347506819705508,-1.0999748867047898,0.77,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.9431208397853792,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.6362379150939101,0.909111664354187,0.72,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-1.553978658753471,-1.8822412662790038,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.54,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-1.990305672302108,-2.0472054438669973,-1.8274572629471952,-1.8894018386217877,-2.175842027962227,-1.610092875190155,-1.0999748867047898,0.49,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.41952842352701486,-1.3873487335150236,-1.8274572629471952,-2.386612848785416,-2.175842027962227,-0.9888405730597928,0.909111664354187,0.52,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.15773221539783266,-1.2223845559270303,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.0895841896214724,-1.0999748867047898,0.58,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.6062666891668145,0.909111664354187,0.78,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(1.0639834225383509,0.9221497527168835,1.674257252879766,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187,0.89,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.2785947981508043,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.03538619531728977,-1.0999748867047898,0.7,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.7685900343659244,-0.7274920231630502,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.1828916405663493,-1.0999748867047898,0.66,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-1.117651645204834,-0.8924562007510436,-0.952028633990455,-0.3977688081309026,0.6101272780073337,0.22008282568037243,-1.0999748867047898,0.67,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.5067938262367422,-0.8924562007510436,-0.0766000050337147,1.0938642223599824,0.6101272780073337,0.06896740083785215,0.909111664354187,0.68,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.016798590021622126,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.8581257305710129,0.909111664354187,0.8,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.41952842352701486,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.32082644224205065,0.909111664354187,0.81,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.36586020086053167,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,0.5055230726051333,0.909111664354187,0.8,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(2.023902852345352,0.7571855751288902,0.7988286239230257,1.5910752325236108,1.167321139201246,1.7816088823864173,0.909111664354187,0.94,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(1.2385142279578056,1.4170422854808635,1.674257252879766,0.5966532121963541,0.6101272780073337,1.1099847719752143,0.909111664354187,0.93,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(1.6748412415064426,1.7469706406568504,1.674257252879766,1.0938642223599824,0.6101272780073337,0.9924505526532535,0.909111664354187,0.92,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(0.6276564089897139,1.087113930304877,1.674257252879766,1.5910752325236108,1.167321139201246,0.8077539222901738,0.909111664354187,0.89,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.24499761810756004,-0.5625278455750569,0.7988286239230257,1.5910752325236108,1.724515000395158,0.7070103057284927,-1.0999748867047898,0.82,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.3322630208172874,0.26229304236491,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.6734291002079332,-1.0999748867047898,0.79,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5933022724298738,-1.0999748867047898,0.58,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-1.4667132560437435,-0.8924562007510436,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.223909011703713,-1.0999748867047898,0.56,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-1.2921824506242887,-1.3873487335150236,-0.952028633990455,-2.386612848785416,-1.618648166768315,-1.056002984100913,-1.0999748867047898,0.56,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.41952842352701486,-1.5523129111030172,-1.8274572629471952,0.09944220203272576,-0.5042604443804907,-0.7034003261350319,0.909111664354187,0.64,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.07046681268810527,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.46833188749111015,0.909111664354187,0.61,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(0.016798590021622126,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,-0.04856681848410872,-1.0999748867047898,0.68,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.5940592289464697,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.11933920911869125,-1.0999748867047898,0.76,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.2785947981508043,2.076898995832837,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.8581257305710129,-1.0999748867047898,0.86,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(1.1512488252480781,1.087113930304877,-0.0766000050337147,1.0938642223599824,1.167321139201246,1.076403566454655,0.909111664354187,0.9,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-1.0303862424951067,0.7571855751288902,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.08575800359813185,-1.0999748867047898,0.71,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.681324631656197,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-1.0056311758200724,-1.0999748867047898,0.62,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.1913293954410769,0.09732886477691666,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.27045463396121155,-1.0999748867047898,0.66,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.4531256035702591,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.2500540516074711,0.909111664354187,0.65,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.5403910062799865,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.08214802400466813,0.909111664354187,0.73,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.3322630208172874,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.2836352571280305,-1.0999748867047898,0.62,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.36586020086053167,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.3376170450023333,0.909111664354187,0.74,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.5403910062799865,0.4272572199529034,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.8413351278107333,0.909111664354187,0.79,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.6734291002079332,0.909111664354187,0.8,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.41952842352701486,0.09732886477691666,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.1157292295252305,-1.0999748867047898,0.69,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.7685900343659244,0.4272572199529034,0.7988286239230257,0.09944220203272576,-0.5042604443804907,0.0018049897967303734,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.23687342844065212,0.909111664354187,0.76,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.9767180198286235,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9756599498929738,0.909111664354187,0.84,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-0.5067938262367422,-0.06763531281107672,0.7988286239230257,1.0938642223599824,1.167321139201246,0.6734291002079332,0.909111664354187,0.78,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.3794478533340162,-1.2223845559270303,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.9384687647789537,-1.0999748867047898,0.67,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.0303862424951067,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-0.7873533399364304,-1.0999748867047898,0.66,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-0.7685900343659244,-0.5625278455750569,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.8880969564981116,-1.0999748867047898,0.65,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.6412440614631982,-1.057420378339037,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.54,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.4667132560437435,-1.3873487335150236,-1.8274572629471952,-2.386612848785416,-1.0614543055744028,-0.9888405730597928,-1.0999748867047898,0.58,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.6276564089897139,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-1.618648166768315,0.3376170450023333,0.909111664354187,0.79,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.889452617118896,0.9221497527168835,0.7988286239230257,0.09944220203272576,-0.5042604443804907,0.15292041463925066,0.909111664354187,0.8,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.016798590021622126,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,0.052933416813421515,-0.1661010378060696,0.909111664354187,0.75,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.5403910062799865,-0.5625278455750569,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.2668446543677508,0.909111664354187,0.73,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.24499761810756004,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.5522849012925116,-1.0999748867047898,0.72,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.0303862424951067,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.7034003261350319,-1.0999748867047898,0.62,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.15773221539783266,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,-0.45154128473083044,-1.0999748867047898,0.67,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.8021872144091686,1.4170422854808635,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.908497538851855,0.909111664354187,0.81,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-0.0766000050337147,-1.3921908284581592,-1.618648166768315,-0.972049970299513,-1.0999748867047898,0.63,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.9030402695923805,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.69,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.6276564089897139,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.2872452367214912,0.909111664354187,0.8,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-2.175842027962227,-1.1903278061831537,-1.0999748867047898,0.43,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.889452617118896,0.9221497527168835,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.10254860635841155,0.909111664354187,0.8,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.5067938262367422,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.8041439426967131,0.909111664354187,0.73,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.7685900343659244,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.6530285178541898,0.909111664354187,0.75,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.1913293954410769,0.09732886477691666,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.0989386267649508,0.909111664354187,0.71,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.08575800359813185,0.909111664354187,0.73,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.7149218116994412,0.5922213975408968,0.7988286239230257,0.5966532121963541,1.167321139201246,0.8581257305710129,0.909111664354187,0.83,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.1913293954410769,0.4272572199529034,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,0.32082644224205065,-1.0999748867047898,0.72,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(1.3257796306675331,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.462587429941097,0.909111664354187,0.94,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.5403910062799865,0.09732886477691666,1.674257252879766,0.5966532121963541,0.6101272780073337,0.23687342844065212,0.909111664354187,0.81,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.6276564089897139,-0.06763531281107672,1.674257252879766,0.09944220203272576,0.6101272780073337,0.10254860635841155,0.909111664354187,0.81,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.2332634488471884,0.909111664354187,0.75,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.8021872144091686,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.27045463396121155,0.909111664354187,0.79,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.7685900343659244,-0.2325994903990701,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.6026567095733507,-1.0999748867047898,0.58,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.0303862424951067,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.7873533399364304,-1.0999748867047898,0.59,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.9030402695923805,-1.8822412662790038,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-2.1138109579985565,-1.0999748867047898,0.47,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.07046681268810527,-1.5523129111030172,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.9626955331560363,-1.0999748867047898,0.49,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.117651645204834,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.610092875190155,-1.0999748867047898,0.47,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.553978658753471,-2.2121696214549904,-1.8274572629471952,-2.386612848785416,-2.7330358891561395,-2.1138109579985565,-1.0999748867047898,0.42,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.2921824506242887,-1.3873487335150236,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-2.2649263828410766,-1.0999748867047898,0.57,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-0.3322630208172874,-1.057420378339037,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.9384687647789537,-1.0999748867047898,0.62,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.10406399273134952,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5522849012925116,0.909111664354187,0.74,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.7149218116994412,0.4272572199529034,0.7988286239230257,0.09944220203272576,0.6101272780073337,0.11933920911869125,0.909111664354187,0.73,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.2049170479145614,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.9048875592583913,0.909111664354187,0.64,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.4667132560437435,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.7201909288953117,-1.0999748867047898,0.63,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.7285094641729257,-1.5523129111030172,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5597210669093144,-1.0999748867047898,0.59,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.016798590021622126,-0.2325994903990701,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8041439426967131,-1.0999748867047898,0.73,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.889452617118896,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.2872452367214912,0.909111664354187,0.79,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.3794478533340162,-0.5625278455750569,-0.952028633990455,0.09944220203272576,0.052933416813421515,-1.1903278061831537,0.909111664354187,0.68,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-0.24499761810756004,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.610092875190155,-1.0999748867047898,0.7,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.36586020086053167,-0.06763531281107672,-0.952028633990455,-1.3921908284581592,-2.175842027962227,-0.2668446543677508,-1.0999748867047898,0.81,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.724515000395158,0.06896740083785215,0.909111664354187,0.85,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.5003104360869879,1.4170422854808635,0.7988286239230257,0.5966532121963541,0.052933416813421515,1.580121649263055,0.909111664354187,0.93,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.8493720469258974,1.2520781078928702,1.674257252879766,1.0938642223599824,1.724515000395158,1.0596129636943752,0.909111664354187,0.91,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.9431208397853792,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.40116947644999135,-1.0999748867047898,0.69,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.5055230726051333,0.909111664354187,0.77,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.1512488252480781,1.087113930304877,0.7988286239230257,1.0938642223599824,-0.5042604443804907,0.9588693471326941,0.909111664354187,0.86,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,1.167321139201246,-0.4347506819705508,0.909111664354187,0.74,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.5067938262367422,-1.5523129111030172,-1.8274572629471952,-2.386612848785416,-1.0614543055744028,-1.9123237248751956,-1.0999748867047898,0.57,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-1.6412440614631982,-2.5420979766309775,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-1.2071184089434333,-1.0999748867047898,0.51,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-1.3794478533340162,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-0.5042604443804907,-0.9552593675392334,0.909111664354187,0.67,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187) +insert into defaults values (1.76210664421617,1.7469706406568504,0.7988286239230257,1.0938642223599824,1.167321139201246,1.7648182796261376,0.909111664354187,0.92,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,-0.06763531281107672,0.7988286239230257,0.5966532121963541,1.167321139201246,0.4551512643242912,0.909111664354187,0.76,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.07046681268810527,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-1.0056311758200724,0.909111664354187,0.72,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,0.11933920911869125,0.909111664354187,0.8,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.24499761810756004,-0.7274920231630502,-0.952028633990455,-1.3921908284581592,-0.5042604443804907,-0.6530285178541898,-1.0999748867047898,0.65,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.1512488252480781,1.2520781078928702,1.674257252879766,1.0938642223599824,-0.5042604443804907,1.244309594057455,0.909111664354187,0.9,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,0.6101272780073337,-0.6698191206144725,0.909111664354187,0.75,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.7685900343659244,-1.057420378339037,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-1.1735372034228724,-1.0999748867047898,0.68,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-1.2921824506242887,-0.8924562007510436,-1.8274572629471952,-1.3921908284581592,-2.175842027962227,-1.0056311758200724,-1.0999748867047898,0.5,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.5403910062799865,0.09732886477691666,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.0018049897967303734,-1.0999748867047898,0.45,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.7149218116994412,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.3340070654088696,0.909111664354187,0.52,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.889452617118896,0.5922213975408968,0.7988286239230257,0.5966532121963541,1.167321139201246,0.6734291002079332,0.909111664354187,0.84,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.9767180198286235,0.7571855751288902,0.7988286239230257,0.5966532121963541,1.167321139201246,0.8413351278107333,0.909111664354187,0.78,-0.5940592289464697,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.8558554370756518,0.26229304236491,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.62,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,-1.618648166768315,-0.6698191206144725,0.909111664354187,0.61,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.24499761810756004,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.5019130930116695,-1.0999748867047898,0.54,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.016798590021622126,-0.06763531281107672,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,0.16971101739953035,-1.0999748867047898,0.66,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.1913293954410769,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.65,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.10406399273134952,0.4272572199529034,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,0.3376170450023333,-1.0999748867047898,0.63,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-1.2049170479145614,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.1661010378060696,-1.0999748867047898,0.62,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,-1.618648166768315,-1.1735372034228724,0.909111664354187,0.64,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.7149218116994412,1.087113930304877,0.7988286239230257,-0.3977688081309026,-1.618648166768315,-0.3340070654088696,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(0.9767180198286235,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.5129592382219361,0.909111664354187,0.94,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(1.5003104360869879,1.9119348182448437,1.674257252879766,1.5910752325236108,1.167321139201246,1.848771293427536,0.909111664354187,0.95,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(1.6748412415064426,1.9119348182448437,1.674257252879766,0.5966532121963541,0.052933416813421515,2.016677321030339,0.909111664354187,0.97,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,1.0428223609340956,0.909111664354187),(2.023902852345352,2.076898995832837,1.674257252879766,1.0938642223599824,1.167321139201246,1.680865265824736,0.909111664354187,0.94,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.4531256035702591,0.26229304236491,1.674257252879766,1.0938642223599824,0.052933416813421515,0.3376170450023333,-1.0999748867047898,0.76,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.6412440614631982,-1.5523129111030172,-0.952028633990455,-1.8894018386217877,-1.0614543055744028,-1.8451613138340752,0.909111664354187,0.44,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.9030402695923805,-2.377133799042984,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-2.348879396642477,-1.0999748867047898,0.46,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-0.5940592289464697,-1.3873487335150236,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-2.180973369039677,-1.0999748867047898,0.54,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.4667132560437435,-1.7172770886910105,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.8377251482172725,0.909111664354187,0.65,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.889452617118896,-0.7274920231630502,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.5019130930116695,0.909111664354187,0.74,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(1.8493720469258974,1.7469706406568504,0.7988286239230257,-0.3977688081309026,1.167321139201246,1.3450532106191362,0.909111664354187,0.91,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(2.023902852345352,1.087113930304877,1.674257252879766,0.5966532121963541,0.6101272780073337,1.680865265824736,0.909111664354187,0.9,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(1.2385142279578056,0.7571855751288902,1.674257252879766,0.5966532121963541,1.724515000395158,2.016677321030339,0.909111664354187,0.94,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(0.2785947981508043,0.4272572199529034,1.674257252879766,1.5910752325236108,1.724515000395158,1.0092411554135332,0.909111664354187,0.88,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.553978658753471,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.6101272780073337,-0.3340070654088696,-1.0999748867047898,0.64,1.6748412415064426,1.9119348182448437,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.7144464713452956,0.909111664354187),(-1.4667132560437435,-0.39756366798706344,-1.8274572629471952,-2.386612848785416,-1.618648166768315,-1.3414432310256739,-1.0999748867047898,0.58,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-1.117651645204834,-0.39756366798706344,-1.8274572629471952,-0.3977688081309026,-2.175842027962227,-1.8451613138340752,-1.0999748867047898,0.52,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.8558554370756518,0.09732886477691666,-0.952028633990455,0.5966532121963541,0.052933416813421515,-1.5093492586284738,-1.0999748867047898,0.48,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.7685900343659244,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.46,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.07046681268810527,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.6698191206144725,0.909111664354187,0.49,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.3322630208172874,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.1661010378060696,0.909111664354187,0.53,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.3257796306675331,1.582006463068857,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.8413351278107333,-1.0999748867047898,0.87,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.8021872144091686,0.9221497527168835,1.674257252879766,1.0938642223599824,0.6101272780073337,1.3450532106191362,0.909111664354187,0.91,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.4531256035702591,0.4272572199529034,1.674257252879766,1.5910752325236108,0.6101272780073337,0.8413351278107333,0.909111664354187,0.88,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.0639834225383509,1.087113930304877,1.674257252879766,0.5966532121963541,1.724515000395158,1.1771471830163363,0.909111664354187,0.86,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(1.9366374496356247,1.9119348182448437,1.674257252879766,1.0938642223599824,0.6101272780073337,1.848771293427536,-1.0999748867047898,0.89,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.36586020086053167,0.4272572199529034,-0.0766000050337147,0.09944220203272576,1.724515000395158,0.42157005880373183,0.909111664354187,0.82,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(0.889452617118896,0.5922213975408968,0.7988286239230257,-0.3977688081309026,0.6101272780073337,-0.3340070654088696,0.909111664354187,0.78,0.36586020086053167,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.0989386267649508,0.909111664354187),(-0.3322630208172874,-1.5523129111030172,-0.0766000050337147,-0.8949798182945309,1.167321139201246,-0.5019130930116695,0.909111664354187,0.76,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.41952842352701486,-1.2223845559270303,-0.952028633990455,-1.8894018386217877,0.052933416813421515,-1.1735372034228724,0.909111664354187,0.56,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.5003104360869879,1.4170422854808635,0.7988286239230257,0.5966532121963541,-0.5042604443804907,-1.0056311758200724,0.909111664354187,0.78,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.6276564089897139,0.7571855751288902,0.7988286239230257,0.5966532121963541,-1.0614543055744028,-0.8377251482172725,0.909111664354187,0.72,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-1.0056311758200724,-1.0999748867047898,0.7,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.2785947981508043,-0.7274920231630502,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-1.5093492586284738,-1.0999748867047898,0.64,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.07046681268810527,-0.8924562007510436,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-2.013067341436875,-1.0999748867047898,0.64,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.6412440614631982,-1.3873487335150236,-0.952028633990455,0.5966532121963541,-1.618648166768315,-1.6772552862312753,-1.0999748867047898,0.46,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.4667132560437435,-1.3873487335150236,-1.8274572629471952,-0.3977688081309026,-1.618648166768315,-3.0205035070536796,0.909111664354187,0.36,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.5067938262367422,-0.5625278455750569,-0.952028633990455,-1.3921908284581592,-1.618648166768315,-0.5019130930116695,-1.0999748867047898,0.42,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.681324631656197,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.8377251482172725,-1.0999748867047898,0.48,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.8558554370756518,-1.057420378339037,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,-0.6698191206144725,-1.0999748867047898,0.47,-0.15773221539783266,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.117651645204834,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.6698191206144725,0.909111664354187,0.54,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.15773221539783266,-0.06763531281107672,-0.952028633990455,0.5966532121963541,-0.5042604443804907,-0.1661010378060696,0.909111664354187,0.56,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.7149218116994412,0.5922213975408968,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.16971101739953035,-1.0999748867047898,0.52,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.09944220203272576,0.052933416813421515,0.5391042781256927,-1.0999748867047898,0.55,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.889452617118896,1.087113930304877,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.7070103057284927,-1.0999748867047898,0.61,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.07046681268810527,-0.06763531281107672,-0.952028633990455,0.09944220203272576,0.052933416813421515,0.06896740083785215,0.909111664354187,0.57,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.10406399273134952,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.6101272780073337,1.0428223609340956,0.909111664354187,0.68,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.9767180198286235,1.2520781078928702,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9420787443724145,0.909111664354187,0.78,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(1.3257796306675331,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.7480276768658578,0.909111664354187,0.94,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(1.6748412415064426,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,1.9495149099892173,0.909111664354187,0.96,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(0.36586020086053167,0.5922213975408968,1.674257252879766,1.5910752325236108,1.724515000395158,1.4290062244205346,0.909111664354187,0.93,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.24499761810756004,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.7405915112490521,0.909111664354187,0.84,-1.117651645204834,-1.057420378339037,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898),(-0.24499761810756004,-0.2325994903990701,-0.0766000050337147,-0.3977688081309026,1.724515000395158,0.5055230726051333,-1.0999748867047898,0.74,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(1.0639834225383509,1.087113930304877,-0.952028633990455,-1.3921908284581592,0.6101272780073337,-0.06535742124438843,0.909111664354187,0.72,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.889452617118896,0.7571855751288902,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.20329222292009272,0.909111664354187,0.74,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.3794478533340162,-1.3873487335150236,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.6362379150939101,-1.0999748867047898,0.64,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.8157748668826532,-2.0472054438669973,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-1.777998902792955,0.909111664354187,0.44,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-1.990305672302108,-2.377133799042984,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-2.0802297524779956,-1.0999748867047898,0.46,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-0.972049970299513,0.909111664354187,0.5,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(2.023902852345352,2.076898995832837,0.7988286239230257,1.5910752325236108,1.724515000395158,1.5129592382219361,0.909111664354187,0.96,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.2785947981508043,0.4272572199529034,1.674257252879766,1.5910752325236108,1.167321139201246,1.0428223609340956,0.909111664354187,0.92,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.4531256035702591,1.2520781078928702,1.674257252879766,0.5966532121963541,1.167321139201246,1.2778907995780144,0.909111664354187,0.92,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(2.023902852345352,1.2520781078928702,1.674257252879766,1.0938642223599824,1.167321139201246,1.4290062244205346,0.909111664354187,0.94,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.1913293954410769,-0.7274920231630502,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.10254860635841155,-1.0999748867047898,0.76,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(-0.15773221539783266,-0.2325994903990701,-0.0766000050337147,1.0938642223599824,0.052933416813421515,-0.30042585988831016,-1.0999748867047898,0.72,-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.9626955331560363,-1.0999748867047898),(0.016798590021622126,-0.06763531281107672,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.535494298532232,-1.0999748867047898,0.66,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.24499761810756004,0.09732886477691666,-0.0766000050337147,1.0938642223599824,0.052933416813421515,-0.7705627371761508,-1.0999748867047898,0.64,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.07046681268810527,0.26229304236491,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.27045463396121155,0.909111664354187,0.74,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(0.10406399273134952,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.6101272780073337,-1.139955997902313,0.909111664354187,0.64,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.553978658753471,-1.7172770886910105,-0.0766000050337147,1.5910752325236108,0.052933416813421515,-1.5765116696695942,-1.0999748867047898,0.38,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.6412440614631982,-1.5523129111030172,-0.952028633990455,0.5966532121963541,-0.5042604443804907,-0.9552593675392334,-1.0999748867047898,0.34,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.3794478533340162,-1.7172770886910105,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-1.2071184089434333,0.909111664354187,0.44,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.2049170479145614,-1.3873487335150236,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-1.5765116696695942,-1.0999748867047898,0.36,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-1.117651645204834,-1.2223845559270303,0.7988286239230257,-1.8894018386217877,-1.0614543055744028,-1.2742808199845537,-1.0999748867047898,0.42,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(-0.9431208397853792,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-1.0056311758200724,-1.0999748867047898,0.48,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(1.2385142279578056,2.076898995832837,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.6062666891668145,0.909111664354187,0.86,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(1.3257796306675331,1.9119348182448437,0.7988286239230257,1.5910752325236108,1.167321139201246,1.076403566454655,0.909111664354187,0.9,-2.3393672831410175,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.7444176972723957,-1.0999748867047898),(0.5403910062799865,0.9221497527168835,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.47194186708457386,0.909111664354187,0.79,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.4531256035702591,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.2332634488471884,0.909111664354187,0.71,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.8041439426967131,-1.0999748867047898,0.64,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.24499761810756004,-0.2325994903990701,-0.952028633990455,0.5966532121963541,0.052933416813421515,-0.585866106813071,-1.0999748867047898,0.62,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.016798590021622126,-0.5625278455750569,-0.952028633990455,1.0938642223599824,0.6101272780073337,-0.2164728460869087,-1.0999748867047898,0.57,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.8021872144091686,0.7571855751288902,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.7573821140093348,0.909111664354187,0.74,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-0.07046681268810527,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.30403583948177093,0.909111664354187,0.69,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,1.167321139201246,0.9756599498929738,0.909111664354187,0.87,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.8493720469258974,1.582006463068857,0.7988286239230257,0.09944220203272576,1.167321139201246,1.4457968271808173,0.909111664354187,0.91,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(1.2385142279578056,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.3114720050985766,0.909111664354187,0.93,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.117651645204834,-0.7274920231630502,1.674257252879766,1.5910752325236108,0.6101272780073337,0.06896740083785215,-1.0999748867047898,0.68,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(-1.0303862424951067,0.09732886477691666,1.674257252879766,-0.3977688081309026,-0.5042604443804907,-0.199682243326629,-1.0999748867047898,0.61,-1.2049170479145614,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.5933022724298738,-1.0999748867047898),(0.36586020086053167,0.26229304236491,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.13612981187897094,0.909111664354187,0.69,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.3794478533340162,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.4347506819705508,0.909111664354187,0.62,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(0.2785947981508043,0.4272572199529034,-0.952028633990455,0.5966532121963541,0.052933416813421515,-0.06535742124438843,-1.0999748867047898,0.72,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5067938262367422,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.2500540516074711,0.909111664354187,0.59,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,0.7988286239230257,1.0938642223599824,1.167321139201246,0.7405915112490521,0.909111664354187,0.66,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.553978658753471,-0.8924562007510436,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.03538619531728977,-1.0999748867047898,0.56,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-2.3393672831410175,-0.5625278455750569,0.7988286239230257,-1.3921908284581592,-1.0614543055744028,-1.9123237248751956,-1.0999748867047898,0.45,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.8157748668826532,-1.3873487335150236,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-2.214554574560236,-1.0999748867047898,0.47,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(0.889452617118896,-0.5625278455750569,1.674257252879766,-0.3977688081309026,0.052933416813421515,0.4047794560434521,0.909111664354187,0.71,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,1.6137028547836172,0.909111664354187,0.94,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(1.5003104360869879,1.9119348182448437,1.674257252879766,1.0938642223599824,1.167321139201246,1.4793780327013768,0.909111664354187,0.94,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,0.7988286239230257,-1.8894018386217877,-1.0614543055744028,-0.40116947644999135,-1.0999748867047898,0.57,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-0.7685900343659244,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.6362379150939101,-1.0999748867047898,0.61,-0.5067938262367422,-1.3873487335150236,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-0.2836352571280305,0.909111664354187),(-1.3794478533340162,-0.2325994903990701,0.7988286239230257,-0.8949798182945309,-0.5042604443804907,-0.2164728460869087,-1.0999748867047898,0.57,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.10254860635841155,0.909111664354187,0.64,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.5403910062799865,0.9221497527168835,-0.0766000050337147,0.5966532121963541,-0.5042604443804907,1.2107283885368956,0.909111664354187,0.85,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.1913293954410769,0.7571855751288902,-0.0766000050337147,-0.8949798182945309,-1.618648166768315,0.18650162015981303,0.909111664354187,0.78,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.8021872144091686,0.7571855751288902,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.8413351278107333,0.909111664354187,0.84,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.4130450333772604,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.2611001968177347,0.909111664354187,0.92,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.9366374496356247,1.087113930304877,1.674257252879766,0.5966532121963541,1.167321139201246,1.9495149099892173,0.909111664354187,0.96,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-1.2049170479145614,-0.39756366798706344,1.674257252879766,1.5910752325236108,1.167321139201246,0.08575800359813185,-1.0999748867047898,0.77,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.681324631656197,-0.39756366798706344,1.674257252879766,0.09944220203272576,0.052933416813421515,-0.06535742124438843,-1.0999748867047898,0.71,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(0.5403910062799865,0.7571855751288902,1.674257252879766,0.5966532121963541,1.167321139201246,0.30403583948177093,-1.0999748867047898,0.79,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(1.4130450333772604,0.9221497527168835,1.674257252879766,0.5966532121963541,0.6101272780073337,1.1435659774957738,0.909111664354187,0.89,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.24499761810756004,0.26229304236491,0.7988286239230257,0.09944220203272576,0.6101272780073337,0.2872452367214912,0.909111664354187,0.82,0.4531256035702591,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4047794560434521,0.909111664354187),(-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,1.5910752325236108,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.76,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(-0.07046681268810527,-1.2223845559270303,-0.952028633990455,-1.8894018386217877,-0.5042604443804907,-0.7369815316555913,0.909111664354187,0.71,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.8021872144091686,1.4170422854808635,-0.952028633990455,1.0938642223599824,-0.5042604443804907,0.8077539222901738,0.909111664354187,0.8,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.10406399273134952,0.26229304236491,-1.8274572629471952,0.09944220203272576,0.052933416813421515,0.8749163333312926,-1.0999748867047898,0.78,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.0639834225383509,0.4272572199529034,-0.952028633990455,0.5966532121963541,-0.5042604443804907,0.9252881416121347,0.909111664354187,0.84,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.3257796306675331,1.7469706406568504,-0.952028633990455,1.0938642223599824,0.052933416813421515,1.2778907995780144,0.909111664354187,0.9,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.2385142279578056,1.2520781078928702,1.674257252879766,0.5966532121963541,0.052933416813421515,1.412215621660255,0.909111664354187,0.92,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(2.023902852345352,2.076898995832837,0.7988286239230257,1.0938642223599824,0.6101272780073337,2.2181645541536983,0.909111664354187,0.97,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.7149218116994412,0.7571855751288902,-0.952028633990455,-0.3977688081309026,0.052933416813421515,0.6062666891668145,0.909111664354187,0.8,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.2785947981508043,0.9221497527168835,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,0.06896740083785215,0.909111664354187,0.81,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(-0.15773221539783266,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-0.199682243326629,-1.0999748867047898,0.75,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(0.8021872144091686,1.087113930304877,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.8581257305710129,0.909111664354187,0.83,0.1913293954410769,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.11933920911869125,0.909111664354187),(1.9366374496356247,1.4170422854808635,0.7988286239230257,0.5966532121963541,0.052933416813421515,2.016677321030339,0.909111664354187,0.96,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.5067938262367422,-0.2325994903990701,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.5690755040527913,0.909111664354187,0.79,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(1.5003104360869879,1.087113930304877,0.7988286239230257,0.5966532121963541,0.6101272780073337,1.3954250188999753,0.909111664354187,0.93,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(1.3257796306675331,1.4170422854808635,1.674257252879766,1.5910752325236108,1.724515000395158,1.1435659774957738,0.909111664354187,0.94,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.36586020086053167,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,0.7741727167696144,0.909111664354187,0.86,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.6276564089897139,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.6101272780073337,0.25366403120093184,-1.0999748867047898,0.79,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(0.8021872144091686,0.09732886477691666,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.4887324698448536,-1.0999748867047898,0.8,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.41952842352701486,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,0.15292041463925066,-1.0999748867047898,0.77,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.15773221539783266,-0.39756366798706344,-0.0766000050337147,-1.3921908284581592,-1.0614543055744028,-0.4347506819705508,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.681324631656197,-0.5625278455750569,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.5690755040527913,-1.0999748867047898,0.65,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.9431208397853792,-0.2325994903990701,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.7705627371761508,-1.0999748867047898,0.61,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-1.7285094641729257,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.1735372034228724,-1.0999748867047898,0.52,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-0.15773221539783266,-0.7274920231630502,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.57,0.6276564089897139,0.4272572199529034,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9252881416121347,0.909111664354187),(-1.6412440614631982,-1.3873487335150236,-1.8274572629471952,-1.8894018386217877,-0.5042604443804907,-1.9123237248751956,-1.0999748867047898,0.53,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.10406399273134952,0.26229304236491,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.1661010378060696,-1.0999748867047898,0.67,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.016798590021622126,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.06535742124438843,-1.0999748867047898,0.68,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.690219702968213,0.909111664354187,0.81,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,1.674257252879766,1.0938642223599824,0.6101272780073337,0.6230572919270941,-1.0999748867047898,0.78,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-1.2921824506242887,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,1.724515000395158,-0.45154128473083044,-1.0999748867047898,0.65,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,-0.5522849012925116,-1.0999748867047898,0.64,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-2.0775710750118352,-1.7172770886910105,-0.952028633990455,-1.3921908284581592,0.6101272780073337,-1.3414432310256739,0.909111664354187,0.64,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.5067938262367422,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.0392123813406318,-1.0999748867047898,0.65,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(-0.41952842352701486,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.9384687647789537,0.909111664354187,0.68,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(1.5003104360869879,1.582006463068857,1.674257252879766,0.5966532121963541,1.167321139201246,0.7909633195298942,0.909111664354187,0.89,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.4531256035702591,0.4272572199529034,0.7988286239230257,0.5966532121963541,1.724515000395158,0.8917069360915754,0.909111664354187,0.86,-1.4667132560437435,-1.2223845559270303,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5690755040527913,-1.0999748867047898),(0.5403910062799865,0.9221497527168835,0.7988286239230257,0.5966532121963541,1.167321139201246,1.0596129636943752,0.909111664354187,0.89,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.36586020086053167,0.5922213975408968,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.6230572919270941,0.909111664354187,0.87,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.2785947981508043,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.052933416813421515,0.4551512643242912,0.909111664354187,0.85,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(1.0639834225383509,1.9119348182448437,0.7988286239230257,1.0938642223599824,1.167321139201246,0.9420787443724145,0.909111664354187,0.9,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.1913293954410769,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,-1.0999748867047898,0.82,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.681324631656197,0.09732886477691666,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.8041439426967131,-1.0999748867047898,0.72,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.8558554370756518,-0.8924562007510436,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.5522849012925116,-1.0999748867047898,0.73,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.4667132560437435,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.7369815316555913,-1.0999748867047898,0.71,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.0303862424951067,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.30042585988831016,-1.0999748867047898,0.71,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-1.2071184089434333,-1.0999748867047898,0.68,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.24499761810756004,0.4272572199529034,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.3376170450023333,-1.0999748867047898,0.75,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(-0.07046681268810527,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.46833188749111015,-1.0999748867047898,0.72,2.023902852345352,0.9221497527168835,0.7988286239230257,1.5910752325236108,1.724515000395158,1.915933704468658,0.909111664354187),(0.889452617118896,0.9221497527168835,0.7988286239230257,1.0938642223599824,1.167321139201246,0.8581257305710129,0.909111664354187,0.89,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.016798590021622126,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.13612981187897094,0.909111664354187,0.84,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.5875758387967152,1.7469706406568504,1.674257252879766,1.0938642223599824,0.052933416813421515,1.412215621660255,0.909111664354187,0.93,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.2385142279578056,1.2520781078928702,1.674257252879766,1.0938642223599824,0.052933416813421515,1.2778907995780144,0.909111664354187,0.93,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.6276564089897139,0.7571855751288902,1.674257252879766,1.5910752325236108,1.724515000395158,0.8077539222901738,0.909111664354187,0.88,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.6276564089897139,0.5922213975408968,1.674257252879766,1.0938642223599824,0.6101272780073337,0.9420787443724145,0.909111664354187,0.9,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.5403910062799865,0.4272572199529034,1.674257252879766,0.5966532121963541,1.724515000395158,0.6398478946873739,0.909111664354187,0.87,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(0.4531256035702591,1.087113930304877,1.674257252879766,1.0938642223599824,0.6101272780073337,0.572685483646252,0.909111664354187,0.86,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(1.6748412415064426,1.7469706406568504,1.674257252879766,1.0938642223599824,1.724515000395158,1.5633310465027752,0.909111664354187,0.94,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.07046681268810527,0.26229304236491,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.27045463396121155,-1.0999748867047898,0.77,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.8558554370756518,-0.06763531281107672,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.1325198322855102,0.909111664354187,0.78,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.9431208397853792,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,-0.5690755040527913,-1.0999748867047898,0.73,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.5940592289464697,-0.2325994903990701,-0.952028633990455,0.09944220203272576,-1.0614543055744028,-0.45154128473083044,-1.0999748867047898,0.73,1.5875758387967152,1.582006463068857,1.674257252879766,1.5910752325236108,1.724515000395158,2.0502585265508984,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.0766000050337147,1.0938642223599824,1.167321139201246,-0.2836352571280305,-1.0999748867047898,0.7,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.3322630208172874,-0.06763531281107672,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.15292041463925066,-1.0999748867047898,0.72,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(0.016798590021622126,-0.7274920231630502,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.0989386267649508,0.909111664354187,0.73,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.15773221539783266,0.4272572199529034,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.2332634488471884,0.909111664354187,0.72,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(2.023902852345352,2.076898995832837,1.674257252879766,1.0938642223599824,1.167321139201246,2.2013739513934185,0.909111664354187,0.97,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(1.5003104360869879,2.076898995832837,1.674257252879766,0.5966532121963541,1.724515000395158,2.1342115403522968,0.909111664354187,0.97,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.6412440614631982,-0.39756366798706344,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.0989386267649508,-1.0999748867047898,0.69,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.9030402695923805,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5933022724298738,-1.0999748867047898,0.57,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.15773221539783266,-1.3873487335150236,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-1.1903278061831537,-1.0999748867047898,0.63,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-0.5940592289464697,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.972049970299513,0.909111664354187,0.66,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.0303862424951067,-0.2325994903990701,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.7369815316555913,-1.0999748867047898,0.64,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(-1.3794478533340162,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.8041439426967131,0.909111664354187,0.68,-1.2921824506242887,-1.057420378339037,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-1.0727935868611929,-1.0999748867047898),(0.7149218116994412,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.7741727167696144,0.909111664354187,0.79,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.9767180198286235,0.4272572199529034,0.7988286239230257,1.5910752325236108,0.6101272780073337,0.908497538851855,0.909111664354187,0.82,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.8493720469258974,2.076898995832837,0.7988286239230257,1.5910752325236108,1.724515000395158,1.7816088823864173,0.909111664354187,0.95,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.4130450333772604,1.9119348182448437,1.674257252879766,1.5910752325236108,1.167321139201246,1.9830961155097766,0.909111664354187,0.96,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.2385142279578056,1.582006463068857,0.7988286239230257,1.0938642223599824,1.724515000395158,1.3786344161396955,0.909111664354187,0.94,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(1.1512488252480781,1.4170422854808635,1.674257252879766,1.5910752325236108,1.167321139201246,1.2778907995780144,0.909111664354187,0.93,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.4531256035702591,0.7571855751288902,0.7988286239230257,1.0938642223599824,1.167321139201246,1.1099847719752143,0.909111664354187,0.91,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.36586020086053167,0.26229304236491,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.8917069360915754,0.909111664354187,0.85,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.6276564089897139,0.4272572199529034,0.7988286239230257,-0.3977688081309026,0.052933416813421515,0.6230572919270941,0.909111664354187,0.84,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.41952842352701486,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.30042585988831016,-1.0999748867047898,0.74,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.3322630208172874,-0.7274920231630502,-0.0766000050337147,0.5966532121963541,0.6101272780073337,0.25366403120093184,-1.0999748867047898,0.76,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(-0.07046681268810527,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.75,-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8377251482172725,-1.0999748867047898),(0.6276564089897139,0.9221497527168835,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.32082644224205065,-1.0999748867047898,0.76,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.7685900343659244,0.26229304236491,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.71,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.0303862424951067,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.6194473123336305,-1.0999748867047898,0.67,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.8157748668826532,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-0.9552593675392334,-1.0999748867047898,0.61,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.9431208397853792,0.4272572199529034,-0.952028633990455,0.09944220203272576,0.6101272780073337,-0.2500540516074711,-1.0999748867047898,0.63,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.41952842352701486,0.4272572199529034,-0.952028633990455,0.09944220203272576,-0.5042604443804907,-0.1157292295252305,-1.0999748867047898,0.64,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.10406399273134952,0.7571855751288902,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.11933920911869125,-1.0999748867047898,0.71,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.6276564089897139,0.5922213975408968,0.7988286239230257,-0.3977688081309026,-0.5042604443804907,0.690219702968213,0.909111664354187,0.82,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.3322630208172874,-0.5625278455750569,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.08575800359813185,-1.0999748867047898,0.73,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.1913293954410769,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.45154128473083044,0.909111664354187,0.74,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-0.41952842352701486,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.5522849012925116,-1.0999748867047898,0.69,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(-1.117651645204834,-1.2223845559270303,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.8880969564981116,-1.0999748867047898,0.64,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(1.1512488252480781,0.9221497527168835,1.674257252879766,1.5910752325236108,0.6101272780073337,1.1939377857766158,0.909111664354187,0.91,-1.8157748668826532,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.3414432310256739,-1.0999748867047898),(0.8021872144091686,0.5922213975408968,1.674257252879766,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187,0.88,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.5966532121963541,1.167321139201246,0.9588693471326941,0.909111664354187,0.85,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(1.0639834225383509,1.087113930304877,1.674257252879766,1.0938642223599824,1.724515000395158,0.9924505526532535,0.909111664354187,0.86,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.5940592289464697,-0.5625278455750569,-0.0766000050337147,-1.3921908284581592,0.052933416813421515,-0.3843788736897117,-1.0999748867047898,0.7,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.1903278061831537,-1.0999748867047898,0.59,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-1.8157748668826532,-1.057420378339037,-1.8274572629471952,-0.8949798182945309,-0.5042604443804907,-1.5429304641490347,-1.0999748867047898,0.6,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.016798590021622126,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.753772134415871,-1.0999748867047898,0.65,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.6276564089897139,1.2520781078928702,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.27045463396121155,0.909111664354187,0.7,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.7149218116994412,1.087113930304877,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.7405915112490521,0.909111664354187,0.76,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.24499761810756004,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.63,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(0.9767180198286235,0.4272572199529034,0.7988286239230257,0.5966532121963541,-1.0614543055744028,0.7070103057284927,0.909111664354187,0.81,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.07046681268810527,-0.39756366798706344,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.22008282568037243,-1.0999748867047898,0.72,0.2785947981508043,0.09732886477691666,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.2668446543677508,0.909111664354187),(-0.5067938262367422,-0.5625278455750569,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.199682243326629,-1.0999748867047898,0.71,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.4551512643242912,0.909111664354187,0.8,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,0.5922213975408968,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.3879888532831724,0.909111664354187,0.77,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,-0.04856681848410872,0.909111664354187,0.74,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.07046681268810527,-1.3873487335150236,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.6734291002079332,-1.0999748867047898,0.7,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.10406399273134952,-1.2223845559270303,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.0989386267649508,0.909111664354187,0.71,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.5875758387967152,1.2520781078928702,0.7988286239230257,1.0938642223599824,1.167321139201246,1.8151900879069767,0.909111664354187,0.93,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.36586020086053167,1.087113930304877,0.7988286239230257,0.5966532121963541,1.724515000395158,0.8749163333312926,-1.0999748867047898,0.85,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.8558554370756518,0.4272572199529034,0.7988286239230257,0.5966532121963541,1.167321139201246,-0.3843788736897117,-1.0999748867047898,0.79,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(-0.681324631656197,-1.3873487335150236,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.06535742124438843,-1.0999748867047898,0.76,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.6276564089897139,-1.2223845559270303,-0.0766000050337147,0.5966532121963541,1.724515000395158,0.06896740083785215,0.909111664354187,0.78,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(0.8021872144091686,-0.8924562007510436,0.7988286239230257,1.5910752325236108,1.724515000395158,0.27045463396121155,0.909111664354187,0.77,-0.24499761810756004,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.6026567095733507,-1.0999748867047898),(1.2385142279578056,1.9119348182448437,0.7988286239230257,1.5910752325236108,1.167321139201246,1.244309594057455,0.909111664354187,0.9,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.889452617118896,0.09732886477691666,1.674257252879766,1.5910752325236108,0.052933416813421515,0.8917069360915754,0.909111664354187,0.87,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.41952842352701486,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,0.6101272780073337,-0.8545157509775522,-1.0999748867047898,0.71,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.7685900343659244,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.40116947644999135,0.909111664354187,0.7,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.6276564089897139,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-2.175842027962227,0.32082644224205065,0.909111664354187,0.7,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.7149218116994412,0.4272572199529034,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,0.27045463396121155,0.909111664354187,0.75,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,-0.8949798182945309,-1.0614543055744028,0.13612981187897094,-1.0999748867047898,0.71,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.41952842352701486,-0.39756366798706344,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.2500540516074711,-1.0999748867047898,0.72,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.24499761810756004,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.7201909288953117,0.909111664354187,0.73,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(0.889452617118896,0.9221497527168835,0.7988286239230257,1.0938642223599824,1.724515000395158,0.908497538851855,-1.0999748867047898,0.83,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.7685900343659244,0.09732886477691666,0.7988286239230257,1.0938642223599824,1.724515000395158,-0.4347506819705508,-1.0999748867047898,0.77,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-0.9431208397853792,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.6362379150939101,0.909111664354187,0.72,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-1.553978658753471,-1.8822412662790038,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.54,0.10406399273134952,-0.2325994903990701,-0.0766000050337147,-1.3921908284581592,-0.5042604443804907,0.08575800359813185,-1.0999748867047898),(-1.990305672302108,-2.0472054438669973,-1.8274572629471952,-1.8894018386217877,-2.175842027962227,-1.610092875190155,-1.0999748867047898,0.49,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.41952842352701486,-1.3873487335150236,-1.8274572629471952,-2.386612848785416,-2.175842027962227,-0.9888405730597928,0.909111664354187,0.52,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.15773221539783266,-1.2223845559270303,-1.8274572629471952,-1.3921908284581592,-1.0614543055744028,-1.0895841896214724,-1.0999748867047898,0.58,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.6062666891668145,0.909111664354187,0.78,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(1.0639834225383509,0.9221497527168835,1.674257252879766,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187,0.89,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.2785947981508043,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.03538619531728977,-1.0999748867047898,0.7,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.7685900343659244,-0.7274920231630502,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.1828916405663493,-1.0999748867047898,0.66,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-1.117651645204834,-0.8924562007510436,-0.952028633990455,-0.3977688081309026,0.6101272780073337,0.22008282568037243,-1.0999748867047898,0.67,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.5067938262367422,-0.8924562007510436,-0.0766000050337147,1.0938642223599824,0.6101272780073337,0.06896740083785215,0.909111664354187,0.68,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.016798590021622126,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.8581257305710129,0.909111664354187,0.8,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(-0.41952842352701486,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.32082644224205065,0.909111664354187,0.81,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(0.36586020086053167,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,0.5055230726051333,0.909111664354187,0.8,0.8021872144091686,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.8749163333312926,0.909111664354187),(2.023902852345352,0.7571855751288902,0.7988286239230257,1.5910752325236108,1.167321139201246,1.7816088823864173,0.909111664354187,0.94,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(1.2385142279578056,1.4170422854808635,1.674257252879766,0.5966532121963541,0.6101272780073337,1.1099847719752143,0.909111664354187,0.93,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(1.6748412415064426,1.7469706406568504,1.674257252879766,1.0938642223599824,0.6101272780073337,0.9924505526532535,0.909111664354187,0.92,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(0.6276564089897139,1.087113930304877,1.674257252879766,1.5910752325236108,1.167321139201246,0.8077539222901738,0.909111664354187,0.89,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.24499761810756004,-0.5625278455750569,0.7988286239230257,1.5910752325236108,1.724515000395158,0.7070103057284927,-1.0999748867047898,0.82,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.3322630208172874,0.26229304236491,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.6734291002079332,-1.0999748867047898,0.79,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.8558554370756518,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5933022724298738,-1.0999748867047898,0.58,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-1.4667132560437435,-0.8924562007510436,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.223909011703713,-1.0999748867047898,0.56,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-1.2921824506242887,-1.3873487335150236,-0.952028633990455,-2.386612848785416,-1.618648166768315,-1.056002984100913,-1.0999748867047898,0.56,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.41952842352701486,-1.5523129111030172,-1.8274572629471952,0.09944220203272576,-0.5042604443804907,-0.7034003261350319,0.909111664354187,0.64,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.07046681268810527,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.46833188749111015,0.909111664354187,0.61,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(0.016798590021622126,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,-1.0614543055744028,-0.04856681848410872,-1.0999748867047898,0.68,0.016798590021622126,-0.5625278455750569,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,0.27045463396121155,-1.0999748867047898),(-0.5940592289464697,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.11933920911869125,-1.0999748867047898,0.76,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.2785947981508043,2.076898995832837,-0.0766000050337147,0.5966532121963541,1.167321139201246,0.8581257305710129,-1.0999748867047898,0.86,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(1.1512488252480781,1.087113930304877,-0.0766000050337147,1.0938642223599824,1.167321139201246,1.076403566454655,0.909111664354187,0.9,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-1.0303862424951067,0.7571855751288902,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.08575800359813185,-1.0999748867047898,0.71,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.681324631656197,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,-1.0614543055744028,-1.0056311758200724,-1.0999748867047898,0.62,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.1913293954410769,0.09732886477691666,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,0.27045463396121155,-1.0999748867047898,0.66,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.4531256035702591,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-0.5042604443804907,-0.2500540516074711,0.909111664354187,0.65,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.5403910062799865,-0.06763531281107672,-0.0766000050337147,0.09944220203272576,0.052933416813421515,-0.08214802400466813,0.909111664354187,0.73,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.3322630208172874,-0.2325994903990701,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-0.2836352571280305,-1.0999748867047898,0.62,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.36586020086053167,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.3376170450023333,0.909111664354187,0.74,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.5403910062799865,0.4272572199529034,-0.0766000050337147,0.5966532121963541,0.052933416813421515,0.8413351278107333,0.909111664354187,0.79,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(0.7149218116994412,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.6101272780073337,0.6734291002079332,0.909111664354187,0.8,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.41952842352701486,0.09732886477691666,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.1157292295252305,-1.0999748867047898,0.69,1.0639834225383509,0.5922213975408968,0.7988286239230257,1.0938642223599824,0.6101272780073337,1.0596129636943752,0.909111664354187),(-0.7685900343659244,0.4272572199529034,0.7988286239230257,0.09944220203272576,-0.5042604443804907,0.0018049897967303734,-1.0999748867047898,0.7,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.23687342844065212,0.909111664354187,0.76,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.9767180198286235,0.09732886477691666,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.9756599498929738,0.909111664354187,0.84,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-0.5067938262367422,-0.06763531281107672,0.7988286239230257,1.0938642223599824,1.167321139201246,0.6734291002079332,0.909111664354187,0.78,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.3794478533340162,-1.2223845559270303,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,-0.9384687647789537,-1.0999748867047898,0.67,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.0303862424951067,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,0.6101272780073337,-0.7873533399364304,-1.0999748867047898,0.66,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-0.7685900343659244,-0.5625278455750569,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.8880969564981116,-1.0999748867047898,0.65,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.6412440614631982,-1.057420378339037,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.54,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(-1.4667132560437435,-1.3873487335150236,-1.8274572629471952,-2.386612848785416,-1.0614543055744028,-0.9888405730597928,-1.0999748867047898,0.58,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.6276564089897139,0.5922213975408968,-0.0766000050337147,-0.8949798182945309,-1.618648166768315,0.3376170450023333,0.909111664354187,0.79,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.889452617118896,0.9221497527168835,0.7988286239230257,0.09944220203272576,-0.5042604443804907,0.15292041463925066,0.909111664354187,0.8,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.016798590021622126,-0.2325994903990701,-0.0766000050337147,0.5966532121963541,0.052933416813421515,-0.1661010378060696,0.909111664354187,0.75,0.6276564089897139,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.7405915112490521,0.909111664354187),(0.5403910062799865,-0.5625278455750569,-0.0766000050337147,0.5966532121963541,0.6101272780073337,-0.2668446543677508,0.909111664354187,0.73,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.24499761810756004,-0.06763531281107672,-0.952028633990455,-0.8949798182945309,0.6101272780073337,-0.5522849012925116,-1.0999748867047898,0.72,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.0303862424951067,-0.8924562007510436,-0.952028633990455,-1.3921908284581592,-1.0614543055744028,-0.7034003261350319,-1.0999748867047898,0.62,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.15773221539783266,-0.5625278455750569,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,-0.45154128473083044,-1.0999748867047898,0.67,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.8021872144091686,1.4170422854808635,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.908497538851855,0.909111664354187,0.81,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.553978658753471,-1.2223845559270303,-0.0766000050337147,-1.3921908284581592,-1.618648166768315,-0.972049970299513,-1.0999748867047898,0.63,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.9030402695923805,-1.057420378339037,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.2406996144639928,-1.0999748867047898,0.69,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.6276564089897139,0.7571855751288902,0.7988286239230257,0.5966532121963541,0.052933416813421515,0.2872452367214912,0.909111664354187,0.8,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-1.7285094641729257,-1.8822412662790038,-0.952028633990455,-0.8949798182945309,-2.175842027962227,-1.1903278061831537,-1.0999748867047898,0.43,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.889452617118896,0.9221497527168835,-0.0766000050337147,0.09944220203272576,-0.5042604443804907,0.10254860635841155,0.909111664354187,0.8,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.5067938262367422,-0.39756366798706344,-0.952028633990455,-0.3977688081309026,-1.618648166768315,-0.8041439426967131,0.909111664354187,0.73,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(-0.7685900343659244,-0.2325994903990701,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.6530285178541898,0.909111664354187,0.75,0.7149218116994412,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,0.8581257305710129,0.909111664354187),(0.1913293954410769,0.09732886477691666,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.0989386267649508,0.909111664354187,0.71,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,0.7988286239230257,1.0938642223599824,0.6101272780073337,0.08575800359813185,0.909111664354187,0.73,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.7149218116994412,0.5922213975408968,0.7988286239230257,0.5966532121963541,1.167321139201246,0.8581257305710129,0.909111664354187,0.83,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.1913293954410769,0.4272572199529034,-0.0766000050337147,-0.3977688081309026,-1.0614543055744028,0.32082644224205065,-1.0999748867047898,0.72,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(1.3257796306675331,1.7469706406568504,1.674257252879766,1.5910752325236108,1.724515000395158,1.462587429941097,0.909111664354187,0.94,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.5403910062799865,0.09732886477691666,1.674257252879766,0.5966532121963541,0.6101272780073337,0.23687342844065212,0.909111664354187,0.81,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.6276564089897139,-0.06763531281107672,1.674257252879766,0.09944220203272576,0.6101272780073337,0.10254860635841155,0.909111664354187,0.81,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.41952842352701486,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.2332634488471884,0.909111664354187,0.75,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(0.8021872144091686,0.4272572199529034,-0.0766000050337147,0.09944220203272576,0.052933416813421515,0.27045463396121155,0.909111664354187,0.79,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.7685900343659244,-0.2325994903990701,-0.0766000050337147,-0.3977688081309026,-0.5042604443804907,-0.6026567095733507,-1.0999748867047898,0.58,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.0303862424951067,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,0.052933416813421515,-0.7873533399364304,-1.0999748867047898,0.59,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.9030402695923805,-1.8822412662790038,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-2.1138109579985565,-1.0999748867047898,0.47,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-0.07046681268810527,-1.5523129111030172,-1.8274572629471952,-1.8894018386217877,-1.618648166768315,-1.9626955331560363,-1.0999748867047898,0.49,1.1512488252480781,1.4170422854808635,0.7988286239230257,1.5910752325236108,1.167321139201246,1.4290062244205346,0.909111664354187),(-1.117651645204834,-1.7172770886910105,-0.952028633990455,-1.8894018386217877,-1.618648166768315,-1.610092875190155,-1.0999748867047898,0.47,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.553978658753471,-2.2121696214549904,-1.8274572629471952,-2.386612848785416,-2.7330358891561395,-2.1138109579985565,-1.0999748867047898,0.42,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.2921824506242887,-1.3873487335150236,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-2.2649263828410766,-1.0999748867047898,0.57,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-0.3322630208172874,-1.057420378339037,-0.0766000050337147,-0.8949798182945309,-0.5042604443804907,-0.9384687647789537,-1.0999748867047898,0.62,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.10406399273134952,-0.06763531281107672,-0.0766000050337147,-0.3977688081309026,0.052933416813421515,-0.5522849012925116,0.909111664354187,0.74,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.7149218116994412,0.4272572199529034,0.7988286239230257,0.09944220203272576,0.6101272780073337,0.11933920911869125,0.909111664354187,0.73,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.2049170479145614,-1.2223845559270303,-0.952028633990455,-0.3977688081309026,0.052933416813421515,-0.9048875592583913,0.909111664354187,0.64,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.4667132560437435,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,-1.0614543055744028,-0.7201909288953117,-1.0999748867047898,0.63,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.7285094641729257,-1.5523129111030172,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-1.5597210669093144,-1.0999748867047898,0.59,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.016798590021622126,-0.2325994903990701,-0.952028633990455,-1.3921908284581592,0.052933416813421515,-0.8041439426967131,-1.0999748867047898,0.73,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(0.889452617118896,0.26229304236491,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.2872452367214912,0.909111664354187,0.79,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-1.3794478533340162,-0.5625278455750569,-0.952028633990455,0.09944220203272576,0.052933416813421515,-1.1903278061831537,0.909111664354187,0.68,-0.41952842352701486,-0.7274920231630502,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.30403583948177093,-1.0999748867047898),(-0.24499761810756004,-0.39756366798706344,-0.952028633990455,-0.8949798182945309,-1.618648166768315,-1.610092875190155,-1.0999748867047898,0.7,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.36586020086053167,-0.06763531281107672,-0.952028633990455,-1.3921908284581592,-2.175842027962227,-0.2668446543677508,-1.0999748867047898,0.81,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.4531256035702591,0.4272572199529034,-0.0766000050337147,0.5966532121963541,1.724515000395158,0.06896740083785215,0.909111664354187,0.85,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.5003104360869879,1.4170422854808635,0.7988286239230257,0.5966532121963541,0.052933416813421515,1.580121649263055,0.909111664354187,0.93,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.8493720469258974,1.2520781078928702,1.674257252879766,1.0938642223599824,1.724515000395158,1.0596129636943752,0.909111664354187,0.91,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.9431208397853792,-0.7274920231630502,-0.952028633990455,-0.8949798182945309,-0.5042604443804907,-0.40116947644999135,-1.0999748867047898,0.69,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.3322630208172874,-0.8924562007510436,-0.0766000050337147,0.09944220203272576,0.6101272780073337,0.5055230726051333,0.909111664354187,0.77,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(1.1512488252480781,1.087113930304877,0.7988286239230257,1.0938642223599824,-0.5042604443804907,0.9588693471326941,0.909111664354187,0.86,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(0.2785947981508043,-0.5625278455750569,-0.0766000050337147,0.09944220203272576,1.167321139201246,-0.4347506819705508,0.909111664354187,0.74,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-0.5067938262367422,-1.5523129111030172,-1.8274572629471952,-2.386612848785416,-1.0614543055744028,-1.9123237248751956,-1.0999748867047898,0.57,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-1.6412440614631982,-2.5420979766309775,-1.8274572629471952,-1.3921908284581592,-1.618648166768315,-1.2071184089434333,-1.0999748867047898,0.51,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187),(-1.3794478533340162,-1.5523129111030172,-1.8274572629471952,-1.3921908284581592,-0.5042604443804907,-0.9552593675392334,0.909111664354187,0.67,1.4130450333772604,1.582006463068857,0.7988286239230257,1.5910752325236108,0.6101272780073337,1.7816088823864173,0.909111664354187); DROP TABLE IF EXISTS model; create table model engine = Memory as select stochasticLinearRegressionState(0.1, 0.0, 5, 'SGD')(target, param1, param2, param3, param4, param5, param6, param7) as state from defaults; with (select state from model) as model select round(evalMLMethod(model, predict1, predict2, predict3, predict4, predict5, predict6, predict7), 12) from defaults; diff --git a/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql b/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql index b27e4275d5d..2ab85d90a66 100644 --- a/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql +++ b/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql @@ -4,7 +4,7 @@ create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDa system stop ttl merges ttl; -insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2) +insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3), (toDateTime('2100-10-10 00:00:00'), 4); select sleep(1) format Null; -- wait if very fast merge happen diff --git a/tests/queries/0_stateless/01034_JSONCompactEachRow.sql b/tests/queries/0_stateless/01034_JSONCompactEachRow.sql index f71597a60e5..e47e5346c01 100644 --- a/tests/queries/0_stateless/01034_JSONCompactEachRow.sql +++ b/tests/queries/0_stateless/01034_JSONCompactEachRow.sql @@ -22,51 +22,64 @@ SELECT 5; /* Check JSONCompactEachRow Input */ CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6]; + SELECT * FROM test_table FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table; SELECT 6; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6]; + SELECT * FROM test_table FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table; SELECT 7; /* Check Nested */ CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; INSERT INTO test_table_2 FORMAT JSONCompactEachRow [16, [15, 16, null], ["first", "second", "third"]]; + SELECT * FROM test_table_2 FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table_2; SELECT 8; /* Check JSONCompactEachRowWithNamesAndTypes and JSONCompactEachRowWithNamesAndTypes Input */ SET input_format_null_as_default = 0; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null]["second", 2, null, 6]; + INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null]["second", 2, null, 6]; + SELECT * FROM test_table FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table; SELECT 9; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null] ["second", 2, null, 6]; + INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null] ["second", 2, null, 6]; + SELECT * FROM test_table FORMAT JSONCompactEachRow; SELECT 10; /* Check Header */ TRUNCATE TABLE test_table; SET input_format_skip_unknown_fields = 1; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", 1, 32]["second", 2, "64"]; + INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "invalid_column"]["first", 1, 32]["second", 2, "64"]; + SELECT * FROM test_table FORMAT JSONCompactEachRow; SELECT 11; TRUNCATE TABLE test_table; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"][1, 2, 3] + INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v4", "v2", "v3"][1, 2, 3] + SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes; SELECT '----------'; SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames; SELECT 12; /* Check Nested */ INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"][16, [15, 16, null], ["first", "second", "third"]]; + INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNames ["v1", "n.id", "n.name"][16, [15, 16, null], ["first", "second", "third"]]; + SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes; SELECT '----------'; SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNames; diff --git a/tests/queries/0_stateless/01060_defaults_all_columns.sql b/tests/queries/0_stateless/01060_defaults_all_columns.sql index afbb01b8cb2..74fad7f75fb 100644 --- a/tests/queries/0_stateless/01060_defaults_all_columns.sql +++ b/tests/queries/0_stateless/01060_defaults_all_columns.sql @@ -3,6 +3,7 @@ DROP TABLE IF EXISTS defaults_all_columns; CREATE TABLE defaults_all_columns (n UInt8 DEFAULT 42, s String DEFAULT concat('test', CAST(n, 'String'))) ENGINE = Memory; INSERT INTO defaults_all_columns FORMAT JSONEachRow {"n": 1, "s": "hello"} {}; + INSERT INTO defaults_all_columns FORMAT JSONEachRow {"n": 2}, {"s": "world"}; SELECT * FROM defaults_all_columns ORDER BY n, s; diff --git a/tests/queries/0_stateless/01072_json_each_row_data_in_square_brackets.sql b/tests/queries/0_stateless/01072_json_each_row_data_in_square_brackets.sql index f7ccc309c5a..ae5e86ec387 100644 --- a/tests/queries/0_stateless/01072_json_each_row_data_in_square_brackets.sql +++ b/tests/queries/0_stateless/01072_json_each_row_data_in_square_brackets.sql @@ -2,9 +2,13 @@ DROP TABLE IF EXISTS json_square_brackets; CREATE TABLE json_square_brackets (id UInt32, name String) ENGINE = Memory; + INSERT INTO json_square_brackets FORMAT JSONEachRow [{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}]; + INSERT INTO json_square_brackets FORMAT JSONEachRow[]; + INSERT INTO json_square_brackets FORMAT JSONEachRow [ ] ; + INSERT INTO json_square_brackets FORMAT JSONEachRow ; SELECT * FROM json_square_brackets ORDER BY id; diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index 534b18a9500..4b8693126a1 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -37,10 +37,10 @@ send -- "INSERT INTO test_01179 values ('foo'); \r" expect "Ok." send -- "INSERT INTO test_01179 values ('foo'); ('bar') \r" -expect "Cannot read data after semicolon" +expect "Syntax error" send -- "SELECT val, count() FROM test_01179 GROUP BY val FORMAT TSV\r" -expect "foo\t2" +expect "foo\t3" send -- "DROP TABLE test_01179\r" expect "Ok." diff --git a/tests/queries/0_stateless/01231_operator_null_in.sql b/tests/queries/0_stateless/01231_operator_null_in.sql index 0424a995b3f..26f342540e3 100644 --- a/tests/queries/0_stateless/01231_operator_null_in.sql +++ b/tests/queries/0_stateless/01231_operator_null_in.sql @@ -112,7 +112,7 @@ DROP TABLE IF EXISTS null_in_subquery; DROP TABLE IF EXISTS null_in_tuple; CREATE TABLE null_in_tuple (dt DateTime, idx int, t Tuple(Nullable(UInt64), Nullable(String))) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL)) +INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL)); SET transform_null_in = 0; diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 713d187cd88..5f82731c54e 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -76,7 +76,7 @@ insert into data_01278 select reinterpretAsString(number), // s6 reinterpretAsString(number), // s7 reinterpretAsString(number) // s8 -from numbers(200000); -- { serverError 241 }" > /dev/null 2>&1 +from numbers(200000);" > /dev/null 2>&1 local ret_code=$? if [[ $ret_code -eq 0 ]]; then diff --git a/tests/queries/0_stateless/01299_alter_merge_tree.sql b/tests/queries/0_stateless/01299_alter_merge_tree.sql index 3c4467926f8..1fa354040f5 100644 --- a/tests/queries/0_stateless/01299_alter_merge_tree.sql +++ b/tests/queries/0_stateless/01299_alter_merge_tree.sql @@ -3,11 +3,11 @@ drop table if exists merge_tree; set allow_deprecated_syntax_for_merge_tree=1; create table merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); -insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) +insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); alter table merge_tree add column dummy String after CounterID; describe table merge_tree; -insert into merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) +insert into merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); select CounterID, dummy from merge_tree where dummy <> '' limit 10; diff --git a/tests/queries/0_stateless/01324_insert_tsv_raw.sql b/tests/queries/0_stateless/01324_insert_tsv_raw.sql index c3812730e5b..4827f34597a 100644 --- a/tests/queries/0_stateless/01324_insert_tsv_raw.sql +++ b/tests/queries/0_stateless/01324_insert_tsv_raw.sql @@ -1,7 +1,6 @@ drop table if exists tsv_raw; create table tsv_raw (strval String, intval Int64, b1 String, b2 String, b3 String, b4 String) engine = Memory; insert into tsv_raw format TSVRaw "a 1 \ \\ "\"" "\\"" -; select * from tsv_raw format TSVRaw; select * from tsv_raw format JSONCompactEachRow; diff --git a/tests/queries/0_stateless/01396_low_cardinality_fixed_string_default.sql b/tests/queries/0_stateless/01396_low_cardinality_fixed_string_default.sql index e31c402559e..f0c89cdf826 100644 --- a/tests/queries/0_stateless/01396_low_cardinality_fixed_string_default.sql +++ b/tests/queries/0_stateless/01396_low_cardinality_fixed_string_default.sql @@ -6,6 +6,7 @@ CREATE TABLE test ) ENGINE = MergeTree() PARTITION BY id ORDER BY id; INSERT INTO test FORMAT CSV 1,RU + INSERT INTO test FORMAT CSV 1, SELECT * FROM test ORDER BY code; diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql index 06374c633e5..52f0eba8ba4 100644 --- a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql @@ -24,51 +24,64 @@ SELECT 5; /* Check JSONCompactStringsEachRow Input */ CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "3"] ["second", "2", "3", "6"]; + SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table; SELECT 6; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; + SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table; SELECT 7; /* Check Nested */ CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; + SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table_2; SELECT 8; /* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Input */ SET input_format_null_as_default = 0; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "3"]["second", "2", "3", "6"]; + INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "3"]["second", "2", "3", "6"]; + SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table; SELECT 9; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; + INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; + SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; SELECT 10; /* Check Header */ TRUNCATE TABLE test_table; SET input_format_skip_unknown_fields = 1; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; + INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "invalid_column"]["first", "1", "32"]["second", "2", "64"]; + SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; SELECT 11; TRUNCATE TABLE test_table; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] + INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v4", "v2", "v3"]["1", "2", "3"] + SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; SELECT '---------'; SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames; SELECT 12; /* Check Nested */ INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; + INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNames ["v1", "n.id", "n.name"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; + SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes; SELECT '---------'; SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNames; diff --git a/tests/queries/0_stateless/01514_input_format_csv_enum_as_number_setting.sql b/tests/queries/0_stateless/01514_input_format_csv_enum_as_number_setting.sql index 526af60434f..9e1783b7ba0 100644 --- a/tests/queries/0_stateless/01514_input_format_csv_enum_as_number_setting.sql +++ b/tests/queries/0_stateless/01514_input_format_csv_enum_as_number_setting.sql @@ -8,6 +8,7 @@ CREATE TABLE table_with_enum_column_for_csv_insert ( SET input_format_csv_enum_as_number = 1; INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2 + SELECT * FROM table_with_enum_column_for_csv_insert; SET input_format_csv_enum_as_number = 0; diff --git a/tests/queries/0_stateless/01514_input_format_json_enum_as_number.sql b/tests/queries/0_stateless/01514_input_format_json_enum_as_number.sql index 0b2cb8c64e2..a5044cd6875 100644 --- a/tests/queries/0_stateless/01514_input_format_json_enum_as_number.sql +++ b/tests/queries/0_stateless/01514_input_format_json_enum_as_number.sql @@ -8,6 +8,7 @@ CREATE TABLE table_with_enum_column_for_json_insert ( ) ENGINE=Memory(); INSERT INTO table_with_enum_column_for_json_insert FORMAT JSONEachRow {"Id":102,"Value":2} + SELECT * FROM table_with_enum_column_for_json_insert; DROP TABLE IF EXISTS table_with_enum_column_for_json_insert; diff --git a/tests/queries/0_stateless/01514_input_format_tsv_enum_as_number_setting.sql b/tests/queries/0_stateless/01514_input_format_tsv_enum_as_number_setting.sql index 033d7d282f0..5ad94eeb20e 100644 --- a/tests/queries/0_stateless/01514_input_format_tsv_enum_as_number_setting.sql +++ b/tests/queries/0_stateless/01514_input_format_tsv_enum_as_number_setting.sql @@ -8,7 +8,9 @@ CREATE TABLE table_with_enum_column_for_tsv_insert ( SET input_format_tsv_enum_as_number = 1; INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2 + INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TabSeparatedRaw 103 1 + SELECT * FROM table_with_enum_column_for_tsv_insert ORDER BY Id; SET input_format_tsv_enum_as_number = 0; diff --git a/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql index 6a0f2a97b4f..4dccfda4bc5 100644 --- a/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql +++ b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql @@ -6,15 +6,23 @@ CREATE TABLE enum_as_num ( ) ENGINE=Memory(); INSERT INTO enum_as_num FORMAT TSV 1 1 + INSERT INTO enum_as_num FORMAT TSV 2 2 + INSERT INTO enum_as_num FORMAT TSV 3 3 + INSERT INTO enum_as_num FORMAT TSV 4 a + INSERT INTO enum_as_num FORMAT TSV 5 b INSERT INTO enum_as_num FORMAT CSV 6,1 + INSERT INTO enum_as_num FORMAT CSV 7,2 + INSERT INTO enum_as_num FORMAT CSV 8,3 + INSERT INTO enum_as_num FORMAT CSV 9,a + INSERT INTO enum_as_num FORMAT CSV 10,b SELECT * FROM enum_as_num ORDER BY Id; diff --git a/tests/queries/0_stateless/01564_test_hint_woes.reference b/tests/queries/0_stateless/01564_test_hint_woes.reference index adb4cc61816..56c07922ccb 100644 --- a/tests/queries/0_stateless/01564_test_hint_woes.reference +++ b/tests/queries/0_stateless/01564_test_hint_woes.reference @@ -20,7 +20,7 @@ select 1; insert into values_01564 values (11); -- { serverError VIOLATED_CONSTRAINT } select nonexistent column; -- { serverError UNKNOWN_IDENTIFIER } -- query after values on the same line -insert into values_01564 values (1); select 1; +insert into values_01564 values (1); select 1; 1 -- a failing insert and then a normal insert (#https://github.com/ClickHouse/ClickHouse/issues/19353) diff --git a/tests/queries/0_stateless/01825_type_json_1.sql b/tests/queries/0_stateless/01825_type_json_1.sql index 6876349677e..dd7c1d2b41f 100644 --- a/tests/queries/0_stateless/01825_type_json_1.sql +++ b/tests/queries/0_stateless/01825_type_json_1.sql @@ -10,6 +10,7 @@ ENGINE = MergeTree ORDER BY tuple(); SYSTEM STOP MERGES t_json; INSERT INTO t_json FORMAT JSONEachRow {"id": 1, "data": {"k1": "aa", "k2": {"k3": "bb", "k4": "c"}}} {"id": 2, "data": {"k1": "ee", "k5": "ff"}}; + INSERT INTO t_json FORMAT JSONEachRow {"id": 3, "data": {"k5":"foo"}}; SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM t_json ORDER BY id; @@ -32,6 +33,7 @@ SELECT '============'; TRUNCATE TABLE t_json; INSERT INTO t_json FORMAT JSONEachRow {"id": 1, "data": {"k1":[{"k2":"aaa","k3":[{"k4":"bbb"},{"k4":"ccc"}]},{"k2":"ddd","k3":[{"k4":"eee"},{"k4":"fff"}]}]}}; + SELECT id, data.k1.k2, data.k1.k3.k4 FROM t_json ORDER BY id; SELECT name, column, type @@ -82,4 +84,5 @@ ORDER BY name; DROP TABLE IF EXISTS t_json; -CREATE TABLE t_json(id UInt64, data Object('JSON')) ENGINE = Log; -- { serverError ILLEGAL_COLUMN } +-- Restore it after it is fixed. +-- CREATE TABLE t_json(id UInt64, data Object('JSON')) ENGINE = Log; -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/01825_type_json_10.sql b/tests/queries/0_stateless/01825_type_json_10.sql index 98f1a766ed8..e13026770f6 100644 --- a/tests/queries/0_stateless/01825_type_json_10.sql +++ b/tests/queries/0_stateless/01825_type_json_10.sql @@ -7,6 +7,7 @@ DROP TABLE IF EXISTS t_json_10; CREATE TABLE t_json_10 (o JSON) ENGINE = Memory; INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}} + INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}} SELECT DISTINCT toTypeName(o) FROM t_json_10; diff --git a/tests/queries/0_stateless/01825_type_json_17.sql b/tests/queries/0_stateless/01825_type_json_17.sql index ee5cf590407..dfbeae2b1fb 100644 --- a/tests/queries/0_stateless/01825_type_json_17.sql +++ b/tests/queries/0_stateless/01825_type_json_17.sql @@ -13,6 +13,7 @@ CREATE FUNCTION hasValidSizes17 AS (arr1, arr2) -> length(arr1) = length(arr2) A SYSTEM STOP MERGES t_json_17; INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} + INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; @@ -24,6 +25,7 @@ SELECT obj.arr.k1.k4 FROM t_json_17 ORDER BY obj.id; TRUNCATE TABLE t_json_17; INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa"}]}]} + INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "bbb", "k3": [{"k4": 10}]}, {"k2": "ccc", "k3": [{"k4": 20}]}]}]} SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; @@ -35,7 +37,9 @@ SELECT obj.arr.k1.k3.k4 FROM t_json_17 ORDER BY obj.id; TRUNCATE TABLE t_json_17; INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k3": "qqq"}, {"k3": "www"}]} + INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "aaa"}], "k3": "eee"}]} + INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 3, "arr": [{"k1": [{"k2": "bbb", "k4": [{"k5": 10}]}, {"k2": "ccc", "k4": [{"k5": 20}]}], "k3": "rrr"}]} SELECT toTypeName(obj) FROM t_json_17 LIMIT 1; diff --git a/tests/queries/0_stateless/01825_type_json_18.sql b/tests/queries/0_stateless/01825_type_json_18.sql index b493982a12c..26bab5ff9e5 100644 --- a/tests/queries/0_stateless/01825_type_json_18.sql +++ b/tests/queries/0_stateless/01825_type_json_18.sql @@ -8,9 +8,11 @@ CREATE TABLE t_json_2(id UInt64, data Object('JSON')) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": 1}}; + SELECT id, data, toTypeName(data) FROM t_json_2 ORDER BY id; TRUNCATE TABLE t_json_2; INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": [1, 2]}}; + SELECT id, data, toTypeName(data) FROM t_json_2 ORDER BY id; diff --git a/tests/queries/0_stateless/01825_type_json_9.sql b/tests/queries/0_stateless/01825_type_json_9.sql index 8fa4b335578..8f6bf2ff205 100644 --- a/tests/queries/0_stateless/01825_type_json_9.sql +++ b/tests/queries/0_stateless/01825_type_json_9.sql @@ -7,6 +7,7 @@ SET allow_experimental_object_type = 1; CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id; INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}}; + INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}}; OPTIMIZE TABLE t_json FINAL; diff --git a/tests/queries/0_stateless/01825_type_json_in_array.sql b/tests/queries/0_stateless/01825_type_json_in_array.sql index e5c20d7ba6b..fc89bad05d4 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_type_json_in_array.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS t_json_array; CREATE TABLE t_json_array (id UInt32, arr Array(JSON)) ENGINE = MergeTree ORDER BY id; INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": 1, "k2": {"k3": 2, "k4": 3}}, {"k1": 2, "k2": {"k5": "foo"}}]} + INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": 3, "k2": {"k3": 4, "k4": 5}}]} SET output_format_json_named_tuples_as_objects = 1; @@ -18,6 +19,7 @@ SELECT toTypeName(arr) FROM t_json_array LIMIT 1; TRUNCATE TABLE t_json_array; INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} + INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; diff --git a/tests/queries/0_stateless/01825_type_json_insert_select.sql b/tests/queries/0_stateless/01825_type_json_insert_select.sql index cd0b280c360..0c83a35cce3 100644 --- a/tests/queries/0_stateless/01825_type_json_insert_select.sql +++ b/tests/queries/0_stateless/01825_type_json_insert_select.sql @@ -9,12 +9,14 @@ CREATE TABLE type_json_src (id UInt32, data JSON) ENGINE = MergeTree ORDER BY id CREATE TABLE type_json_dst AS type_json_src; INSERT INTO type_json_src VALUES (1, '{"k1": 1, "k2": "foo"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src; SELECT DISTINCT toTypeName(data) FROM type_json_dst; SELECT id, data FROM type_json_dst ORDER BY id; INSERT INTO type_json_src VALUES (2, '{"k1": 2, "k2": "bar"}') (3, '{"k1": 3, "k3": "aaa"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id > 1; SELECT DISTINCT toTypeName(data) FROM type_json_dst; @@ -23,10 +25,12 @@ SELECT id, data FROM type_json_dst ORDER BY id; INSERT INTO type_json_dst VALUES (4, '{"arr": [{"k11": 5, "k22": 6}, {"k11": 7, "k33": 8}]}'); INSERT INTO type_json_src VALUES (5, '{"arr": "not array"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; -- { serverError INCOMPATIBLE_COLUMNS } TRUNCATE TABLE type_json_src; -INSERT INTO type_json_src VALUES (5, '{"arr": [{"k22": "str1"}]}') +INSERT INTO type_json_src VALUES (5, '{"arr": [{"k22": "str1"}]}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; SELECT DISTINCT toTypeName(data) FROM type_json_dst; @@ -45,6 +49,7 @@ SET max_insert_threads = 1; SET output_format_json_named_tuples_as_objects = 1; INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; + INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; INSERT INTO type_json_dst SELECT data FROM type_json_src; @@ -57,6 +62,7 @@ TRUNCATE TABLE type_json_dst; -- Insert in another order. Order is important, because a way how defaults are filled differs. INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; + INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; INSERT INTO type_json_dst SELECT data FROM type_json_src; diff --git a/tests/queries/0_stateless/01925_json_as_string_data_in_square_brackets.sql b/tests/queries/0_stateless/01925_json_as_string_data_in_square_brackets.sql index 3b1e5176ff9..6e6f306c109 100644 --- a/tests/queries/0_stateless/01925_json_as_string_data_in_square_brackets.sql +++ b/tests/queries/0_stateless/01925_json_as_string_data_in_square_brackets.sql @@ -3,8 +3,11 @@ DROP TABLE IF EXISTS json_square_brackets; CREATE TABLE json_square_brackets (field String) ENGINE = Memory; INSERT INTO json_square_brackets FORMAT JSONAsString [{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}]; + INSERT INTO json_square_brackets FORMAT JSONAsString[]; + INSERT INTO json_square_brackets FORMAT JSONAsString [ ] ; + INSERT INTO json_square_brackets FORMAT JSONEachRow ; SELECT * FROM json_square_brackets; diff --git a/tests/queries/0_stateless/01943_query_id_check.sql b/tests/queries/0_stateless/01943_query_id_check.sql index ad9e88e0478..74c2dad8897 100644 --- a/tests/queries/0_stateless/01943_query_id_check.sql +++ b/tests/queries/0_stateless/01943_query_id_check.sql @@ -16,7 +16,7 @@ SELECT query FROM system.query_log WHERE initial_query_id = (SELECT * FROM tmp) DROP TABLE tmp; CREATE TABLE tmp (str String) ENGINE = Log; -INSERT INTO tmp (*) VALUES ('a') +INSERT INTO tmp (*) VALUES ('a'); SELECT count() FROM (SELECT initialQueryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); SELECT count() FROM (SELECT queryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); SELECT count() FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID() HAVING t == initialQueryID()); diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.sql b/tests/queries/0_stateless/02002_parse_map_int_key.sql index 8d009037dd7..c4b48e3d2b6 100644 --- a/tests/queries/0_stateless/02002_parse_map_int_key.sql +++ b/tests/queries/0_stateless/02002_parse_map_int_key.sql @@ -2,7 +2,6 @@ DROP TABLE IF EXISTS t_map_int_key; CREATE TABLE t_map_int_key (m1 Map(UInt32, UInt32), m2 Map(Date, UInt32)) ENGINE = Memory; INSERT INTO t_map_int_key FORMAT CSV "{1:2, 3: 4, 5 :6, 7 : 8}","{'2021-05-20':1, '2021-05-21': 2, '2021-05-22' :3, '2021-05-23' : 4}" -; SELECT m1, m2 FROM t_map_int_key; diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect index 7e78b7e7280..f09e9613a87 100755 --- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect +++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect @@ -33,7 +33,6 @@ expect "Syntax error: " expect ":) " send -- " \\l ; \\d; \r" -expect "Syntax error (Multi-statements are not allowed): " expect ":) " send -- " \\l ;\r" diff --git a/tests/queries/0_stateless/02114_bool_type.sql b/tests/queries/0_stateless/02114_bool_type.sql index d4ea4e54028..6ff9206bb59 100644 --- a/tests/queries/0_stateless/02114_bool_type.sql +++ b/tests/queries/0_stateless/02114_bool_type.sql @@ -15,22 +15,31 @@ SELECT value,f FROM bool_test where value > 0; set bool_true_representation='True'; set bool_false_representation='False'; + INSERT INTO bool_test (value,f) FORMAT CSV True,test + INSERT INTO bool_test (value,f) FORMAT TSV False test + SELECT value,f FROM bool_test order by value FORMAT CSV; SELECT value,f FROM bool_test order by value FORMAT TSV; set bool_true_representation='Yes'; set bool_false_representation='No'; + INSERT INTO bool_test (value,f) FORMAT CSV Yes,test + INSERT INTO bool_test (value,f) FORMAT TSV No test + SELECT value,f FROM bool_test order by value FORMAT CSV; SELECT value,f FROM bool_test order by value FORMAT TSV; set bool_true_representation='On'; set bool_false_representation='Off'; + INSERT INTO bool_test (value,f) FORMAT CSV On,test + INSERT INTO bool_test (value,f) FORMAT TSV Off test + SELECT value,f FROM bool_test order by value FORMAT CSV; SELECT value,f FROM bool_test order by value FORMAT TSV; diff --git a/tests/queries/0_stateless/02184_ipv6_select_parsing.sql b/tests/queries/0_stateless/02184_ipv6_select_parsing.sql index 2892de309c4..ba5399555f8 100644 --- a/tests/queries/0_stateless/02184_ipv6_select_parsing.sql +++ b/tests/queries/0_stateless/02184_ipv6_select_parsing.sql @@ -2,8 +2,11 @@ drop table if exists ips_v6; create table ips_v6(i IPv6) Engine=Memory; INSERT INTO ips_v6 SELECT toIPv6('::ffff:127.0.0.1'); + INSERT INTO ips_v6 values ('::ffff:127.0.0.1'); + INSERT INTO ips_v6 FORMAT TSV ::ffff:127.0.0.1 + INSERT INTO ips_v6 SELECT ('::ffff:127.0.0.1'); SELECT * FROM ips_v6; diff --git a/tests/queries/0_stateless/02192_comment_error.reference b/tests/queries/0_stateless/02192_comment_error.reference index 21da4d2be9e..5f097eb8df6 100644 --- a/tests/queries/0_stateless/02192_comment_error.reference +++ b/tests/queries/0_stateless/02192_comment_error.reference @@ -1,5 +1,5 @@ -OK -OK +FAIL +FAIL OK OK OK diff --git a/tests/queries/0_stateless/02192_comment_error.sh b/tests/queries/0_stateless/02192_comment_error.sh index 78ff474ae84..d48c95a6419 100755 --- a/tests/queries/0_stateless/02192_comment_error.sh +++ b/tests/queries/0_stateless/02192_comment_error.sh @@ -4,8 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# just stop, no exception ${CLICKHOUSE_CLIENT} --query="#" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query="#not a comemnt" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL' +# syntax error ${CLICKHOUSE_CLIENT} --query="select 1 #not a comemnt" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query="select 1 #" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "select 42 #" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql index 86db9d04aa0..a232a4196db 100644 --- a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql +++ b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql @@ -6,9 +6,7 @@ DROP TABLE IF EXISTS t_async_insert_02193_1; CREATE TABLE t_async_insert_02193_1 (id UInt32, s String) ENGINE = Memory; -INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV -1,aaa -; +INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV 1,aaa INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb'); diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.reference b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.reference index d3605d15a3d..2c94e483710 100644 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.reference +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.reference @@ -1,2 +1,2 @@ OK -Missing columns +OK diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh index eb4a91bd850..a6c47d80fa9 100755 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||: -$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0 -- { serverError 47 }" 2>&1 | grep -o 'Missing columns' +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02242_subcolumns_sizes.sql b/tests/queries/0_stateless/02242_subcolumns_sizes.sql index 8c3d8e69238..d29241131d3 100644 --- a/tests/queries/0_stateless/02242_subcolumns_sizes.sql +++ b/tests/queries/0_stateless/02242_subcolumns_sizes.sql @@ -9,6 +9,7 @@ ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t_subcolumns_sizes FORMAT JSONEachRow {"id": 1, "arr": [1, 2, 3], "n": null, "d": {"k1": "v1", "k2": [{"k3": 1, "k4": "v2"}, {"k3": 3}]}} + INSERT INTO t_subcolumns_sizes FORMAT JSONEachRow {"id": 2, "arr": [0], "n": "foo", "d": {"k1": "v3", "k2": [{"k4": "v4"}, {"k3": "v5", "k5": 5}]}} OPTIMIZE TABLE t_subcolumns_sizes FINAL; diff --git a/tests/queries/0_stateless/02249_insert_select_from_input_schema_inference.sql b/tests/queries/0_stateless/02249_insert_select_from_input_schema_inference.sql index eb83309f117..a0c6701faae 100644 --- a/tests/queries/0_stateless/02249_insert_select_from_input_schema_inference.sql +++ b/tests/queries/0_stateless/02249_insert_select_from_input_schema_inference.sql @@ -3,5 +3,6 @@ set use_structure_from_insertion_table_in_table_functions = 1; drop table if exists test_02249; create table test_02249 (x UInt32, y String) engine=Memory(); insert into test_02249 select * from input() format JSONEachRow {"x" : 1, "y" : "string1"}, {"y" : "string2", "x" : 2}; + select * from test_02249; drop table test_02249; diff --git a/tests/queries/0_stateless/02267_jsonlines_ndjson_format.sql b/tests/queries/0_stateless/02267_jsonlines_ndjson_format.sql index 5e32758b328..5dfb17da98c 100644 --- a/tests/queries/0_stateless/02267_jsonlines_ndjson_format.sql +++ b/tests/queries/0_stateless/02267_jsonlines_ndjson_format.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS 02267_t; CREATE TABLE 02267_t (n1 UInt32, n2 UInt32) ENGINE = Memory; INSERT INTO 02267_t FORMAT JSONLines {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6}; + INSERT INTO 02267_t FORMAT NDJSON {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6}; SELECT * FROM 02267_t ORDER BY n1, n2 FORMAT JSONLines; diff --git a/tests/queries/0_stateless/02366_kql_summarize.sql b/tests/queries/0_stateless/02366_kql_summarize.sql index ca16bc3a755..1480d10e14a 100644 --- a/tests/queries/0_stateless/02366_kql_summarize.sql +++ b/tests/queries/0_stateless/02366_kql_summarize.sql @@ -40,7 +40,7 @@ create table Dates EventTime DateTime, ) ENGINE = Memory; -Insert into Dates VALUES ('2015-10-12') , ('2016-10-12') +Insert into Dates VALUES ('2015-10-12') , ('2016-10-12'); Select '-- test summarize --' ; set dialect='kusto'; Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age); diff --git a/tests/queries/0_stateless/02387_parse_date_as_datetime.sql b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql index 9727f677be2..24d367e56b6 100644 --- a/tests/queries/0_stateless/02387_parse_date_as_datetime.sql +++ b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql @@ -1,13 +1,19 @@ CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime); + INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; + INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; + SELECT * FROM test ORDER BY i; DROP TABLE test; CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime64); + INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; + INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; + SELECT * FROM test ORDER BY i; DROP TABLE test; diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 776d1f850b0..a0312b7a40c 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -25,7 +25,7 @@ function insert_data $CLICKHOUSE_CURL -sS -d 'begin transaction' "$CLICKHOUSE_URL&$TXN_SETTINGS" SETTINGS="$SETTINGS&session_check=1" BEGIN="begin transaction;" - COMMIT=$(echo -ne "\n\ncommit") + COMMIT=$(echo -ne "\n\n\ncommit") fi # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql index 71a2381d7b6..845f895b7b4 100644 --- a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql @@ -7,21 +7,22 @@ create table test (x Nullable(UInt32), y UInt32) engine=Memory(); set use_structure_from_insertion_table_in_table_functions=2; set input_format_json_infer_incomplete_types_as_strings=0; + insert into test select * from file(02458_data.jsonl); insert into test select x, 1 from file(02458_data.jsonl); insert into test select x, y from file(02458_data.jsonl); insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x, z from file(02458_data.jsonl); - insert into test select * from file(02458_data.jsoncompacteachrow); insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} - insert into test select * from input() format CSV 1,2 + insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} -insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42} + +insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42}; select * from test order by y; @@ -31,9 +32,9 @@ insert into test select * from file(02458_data.jsonl); insert into test select x from file(02458_data.jsonl); insert into test select y from file(02458_data.jsonl); insert into test select y as x from file(02458_data.jsonl); - insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} -insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42} + +insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42}; select * from test order by x; diff --git a/tests/queries/0_stateless/02495_analyzer_storage_join.sql b/tests/queries/0_stateless/02495_analyzer_storage_join.sql index aeab15862c3..7e6c03971f9 100644 --- a/tests/queries/0_stateless/02495_analyzer_storage_join.sql +++ b/tests/queries/0_stateless/02495_analyzer_storage_join.sql @@ -23,7 +23,7 @@ SELECT x FROM t1 ALL RIGHT JOIN tj ON t1.id1 == tj.key1 AND t1.id2 == tj.key2 OR SELECT '--- name clashes ---'; CREATE TABLE t (key2 UInt64, key1 Int64, b UInt64, x UInt64, val UInt64) ENGINE = Memory; -INSERT INTO t VALUES (1, -1, 11, 111, 1111), (2, -2, 22, 222, 2222), (3, -3, 33, 333, 2222), (4, -4, 44, 444, 4444), (5, -5, 55, 555, 5555) +INSERT INTO t VALUES (1, -1, 11, 111, 1111), (2, -2, 22, 222, 2222), (3, -3, 33, 333, 2222), (4, -4, 44, 444, 4444), (5, -5, 55, 555, 5555); SELECT '-- using --'; diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql index 487b6e7f58e..604c5b179cf 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql @@ -14,12 +14,12 @@ CREATE TABLE regexp_dictionary_source_table -- test back reference. -INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) -INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']) -INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']) -INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']) -INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']) +INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']); +INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']); +INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']); +INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']); +INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']); +INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']); create dictionary regexp_dict1 ( @@ -50,23 +50,24 @@ CREATE TABLE needle_table ENGINE=TinyLog; INSERT INTO needle_table select concat(toString(number + 30), '/tclwebkit', toString(number)) from system.numbers limit 15; + select * from needle_table; select dictGet(regexp_dict1, ('name', 'version'), key) from needle_table; -- test invalid -INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']) +INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']); SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION } truncate table regexp_dictionary_source_table; -INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']) +INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']); SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION } truncate table regexp_dictionary_source_table; -INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) -INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']); +INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']); +INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']); SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION } -- test priority diff --git a/tests/queries/0_stateless/02536_replace_with_nonconst_needle_and_replacement.sql b/tests/queries/0_stateless/02536_replace_with_nonconst_needle_and_replacement.sql index 926bde3a74b..b88224a89c1 100644 --- a/tests/queries/0_stateless/02536_replace_with_nonconst_needle_and_replacement.sql +++ b/tests/queries/0_stateless/02536_replace_with_nonconst_needle_and_replacement.sql @@ -7,7 +7,7 @@ CREATE TABLE test_tab engine = MergeTree() ORDER BY id; -INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'xx') (2, 'Hello World', 'll', 'x') (3, 'Hello World', 'not_found', 'x') (4, 'Hello World', '[eo]', 'x') (5, 'Hello World', '.', 'x') +INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'xx') (2, 'Hello World', 'll', 'x') (3, 'Hello World', 'not_found', 'x') (4, 'Hello World', '[eo]', 'x') (5, 'Hello World', '.', 'x'); SELECT '** replaceAll() **'; @@ -77,7 +77,7 @@ CREATE TABLE test_tab engine = MergeTree() ORDER BY id; -INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'x') (2, 'Hello World', '', 'y') +INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'x') (2, 'Hello World', '', 'y'); -- needle: non-const, replacement: const SELECT replaceAll(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND } diff --git a/tests/queries/0_stateless/02553_type_json_attach_partition.sql b/tests/queries/0_stateless/02553_type_json_attach_partition.sql index 9225106f767..e77f5885ec3 100644 --- a/tests/queries/0_stateless/02553_type_json_attach_partition.sql +++ b/tests/queries/0_stateless/02553_type_json_attach_partition.sql @@ -5,6 +5,7 @@ DROP TABLE IF EXISTS t_json_attach_partition; CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}}; + ALTER TABLE t_json_attach_partition DETACH PARTITION tuple(); INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": [1, 2]}}; diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index e393d96fc40..97d644fa4d6 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -11,9 +11,13 @@ SET async_insert_use_adaptive_busy_timeout=0; SET async_insert_busy_timeout_max_ms = 10000000; INSERT INTO t_async_inserts_flush VALUES (1) (2); + INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 10} {"a": 20}; -INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"} -INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200} + +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"}; + +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200}; + INSERT INTO t_async_inserts_flush VALUES (3) (4) (5); SELECT sleep(1) FORMAT Null; diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 4b0c199ed35..2e55712e49c 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -17,8 +17,6 @@ Bad arguments Bad arguments Bad arguments Bad arguments -Bad arguments -Bad arguments 320 317 Bad arguments diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index fea2978c116..7b959a3c3dc 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -22,14 +22,17 @@ $CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 $CLICKHOUSE_LOCAL -n "SELECT 301" $CLICKHOUSE_LOCAL -n "SELECT 302;" $CLICKHOUSE_LOCAL -n "SELECT 304;SELECT 305;" +# --multiquery and -n are obsolete by now and no-ops. +# The only exception is a single --multiquery "" $CLICKHOUSE_LOCAL --multiquery --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL -n --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery -n 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery --multiquery "SELECT 306; SELECT 307;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL -n --multiquery "SELECT 307; SELECT 308;" 2>&1 | grep -o 'Bad arguments' -$CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery "SELECT 311;" --multiquery "SELECT 312;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery "SELECT 313;" -n "SELECT 314;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL -n "SELECT 320" --query "SELECT 317;" +# --query should be followed by SQL $CLICKHOUSE_LOCAL --query -n "SELECT 400;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --query -n --multiquery "SELECT 401;" 2>&1 | grep -o 'Bad arguments' diff --git a/tests/queries/0_stateless/02771_multiple_query_arguments.reference b/tests/queries/0_stateless/02771_multiple_query_arguments.reference index 0c008e96965..b98ad7086a9 100644 --- a/tests/queries/0_stateless/02771_multiple_query_arguments.reference +++ b/tests/queries/0_stateless/02771_multiple_query_arguments.reference @@ -2,15 +2,17 @@ 101 202 202 -Multi-statements are not allowed -Empty query +303 +303 +303 Bad arguments Syntax error 101 101 202 202 -Multi-statements are not allowed -Empty query +303 +303 +303 Bad arguments Syntax error diff --git a/tests/queries/0_stateless/02771_multiple_query_arguments.sh b/tests/queries/0_stateless/02771_multiple_query_arguments.sh index 2958c9e9a72..ae6e23eb61a 100755 --- a/tests/queries/0_stateless/02771_multiple_query_arguments.sh +++ b/tests/queries/0_stateless/02771_multiple_query_arguments.sh @@ -7,15 +7,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # clickhouse-client $CLICKHOUSE_CLIENT --query "SELECT 101" --query "SELECT 101" $CLICKHOUSE_CLIENT --query "SELECT 202;" --query "SELECT 202;" -$CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 | grep -o 'Multi-statements are not allowed' -$CLICKHOUSE_CLIENT --query "" --query "" 2>&1 | grep -o 'Empty query' +$CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELECT 303; SELECT 303" +$CLICKHOUSE_CLIENT --query "" --query "" 2>&1 $CLICKHOUSE_CLIENT --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error' # clickhouse-local $CLICKHOUSE_LOCAL --query "SELECT 101" --query "SELECT 101" $CLICKHOUSE_LOCAL --query "SELECT 202;" --query "SELECT 202;" -$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 | grep -o 'Multi-statements are not allowed' # behaves differently than clickhouse-client, TODO make it consistent -$CLICKHOUSE_LOCAL --query "" --query "" 2>&1 | grep -o 'Empty query' # behaves equally different than clickhouse-client TODO +$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" +$CLICKHOUSE_LOCAL --query "" --query "" $CLICKHOUSE_LOCAL --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error' diff --git a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.reference b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.reference index d99881dfa6b..1cc8f10a4cf 100644 --- a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.reference +++ b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.reference @@ -1,5 +1,7 @@ -Multi-statements are not allowed -Multi-statements are not allowed +1 +2 +1 +2 1 2 1 diff --git a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh index e2dab337dc2..3f2b732e71b 100755 --- a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh +++ b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh @@ -4,10 +4,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# throw exception -$CLICKHOUSE_CLIENT -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed' -$CLICKHOUSE_LOCAL -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed' -# execute correctly +# clickhouse-local and clickhouse-client behave the same +$CLICKHOUSE_CLIENT -q "select 1; select 2;" +$CLICKHOUSE_LOCAL -q "select 1; select 2;" + +# -n is a no-op $CLICKHOUSE_CLIENT -n -q "select 1; select 2;" $CLICKHOUSE_LOCAL -n -q "select 1; select 2;" diff --git a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference index 250a673a26b..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference +++ b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference @@ -1,6 +0,0 @@ -client no multiquery -Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0) -client multiquery -local no multiquery -Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0) -local multiquery diff --git a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh index 8164c91b2ae..2b9ac6bfd54 100755 --- a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh +++ b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh @@ -4,12 +4,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo "client no multiquery" -$CLICKHOUSE_CLIENT -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)" -echo "client multiquery" -$CLICKHOUSE_CLIENT -n -q "insert into function null() values (1); -- { foo }" - -echo "local no multiquery" -$CLICKHOUSE_LOCAL -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)" -echo "local multiquery" -$CLICKHOUSE_LOCAL -n -q "insert into function null() values (1); -- { foo }" +$CLICKHOUSE_CLIENT -q "insert into function null() values (1); -- { foo }" +$CLICKHOUSE_LOCAL -q "insert into function null() values (1); -- { foo }" diff --git a/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql b/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql index 0b053d3e870..6919e23ad1f 100644 --- a/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql +++ b/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql @@ -18,7 +18,7 @@ ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (key, id, time); -INSERT INTO test1 VALUES ('id0', now(), 3, false) +INSERT INTO test1 VALUES ('id0', now(), 3, false); SELECT last_value(value) OVER (PARTITION BY id ORDER BY time ASC) as last_value FROM test1 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 9081035579d..6ffc4064413 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -195,7 +195,7 @@ $CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq " drop table d; truncate src; - insert into src values (1) + insert into src values (1); create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do @@ -213,6 +213,7 @@ done # we wait for a slow refresh, not a previous fast one.) $CLICKHOUSE_CLIENT -nq " insert into src select * from numbers(1000) settings max_block_size=1; + system start view e;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] do diff --git a/tests/queries/0_stateless/02950_dictionary_short_circuit.sql b/tests/queries/0_stateless/02950_dictionary_short_circuit.sql index 12c934a8d2d..c613709a975 100644 --- a/tests/queries/0_stateless/02950_dictionary_short_circuit.sql +++ b/tests/queries/0_stateless/02950_dictionary_short_circuit.sql @@ -241,12 +241,12 @@ CREATE TABLE regexp_dictionary_source_table values Array(String), ) ENGINE=TinyLog; -INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) -INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']) -INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']) -INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']) -INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']) +INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']); +INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']); +INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']); +INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']); +INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']); +INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']); DROP DICTIONARY IF EXISTS regexp_dict; create dictionary regexp_dict diff --git a/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql b/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql index 14823644b96..30fb3c76c1f 100644 --- a/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql +++ b/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; CREATE TABLE t2 (id Int32, pe Map(String, Tuple(a UInt64, b UInt64))) ENGINE = MergeTree ORDER BY id; -INSERT INTO t2 VALUES (1, {'a': (1, 2), 'b': (2, 3)}), +INSERT INTO t2 VALUES (1, {'a': (1, 2), 'b': (2, 3)}),; CREATE TABLE t3 (id Int32, c Tuple(v String, pe Map(String, Tuple(a UInt64, b UInt64)))) ENGINE = MergeTree ORDER BY id; INSERT INTO t3 VALUES (1, ('A', {'a':(1, 2),'b':(2, 3)})); diff --git a/tests/queries/0_stateless/03005_input_function_in_join.sql b/tests/queries/0_stateless/03005_input_function_in_join.sql index 8a6b4a48a8d..a6fc27cd8d0 100644 --- a/tests/queries/0_stateless/03005_input_function_in_join.sql +++ b/tests/queries/0_stateless/03005_input_function_in_join.sql @@ -8,7 +8,7 @@ SELECT x.number FROM ( ) AS x INNER JOIN input('a UInt64') AS y ON x.number = y.a Format CSV 2 -; + select * from test; drop table test; diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.reference b/tests/queries/0_stateless/03156_default_multiquery_split.reference new file mode 100644 index 00000000000..0f3a1baff45 --- /dev/null +++ b/tests/queries/0_stateless/03156_default_multiquery_split.reference @@ -0,0 +1,10 @@ +Syntax error +101 +102 +1 +2; +3 +4 +7 +8 +9 diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh new file mode 100755 index 00000000000..ac64c2d093d --- /dev/null +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-ordinary-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SQL_FILE_NAME=$"03156_default_multiquery_split_$$.sql" + +# The old multiquery implementation uses '\n' to split INSERT query segmentation +# this case is mainly to test the following situations +# 1. INSERT(format=Values) query: split by ';' +# 2. INSERT(format is not Values) query: split by '\n\n' instead of ';', then discard the ramaining data +# 3. INSERT(format is not Values) query: split by '\n\n', causing the trailing ';'' to be treated as the part of last value +# 4. The client uses multiquery by default, regardless of whether multiquery option is used. + +# create table test1, test2, then +# 1. insert 101, 102 into test1 +# 2. insert 1, 2; into test2, ';' will be treated as a part of a value +# 3. insert 3, 4; '6' will be treated as the next query because of the empty line, we use empty line to determine the end of insert query(format IS NOT VALUES) +# '6' will cause Syntax error +cat << EOF > "$SQL_FILE_NAME" +DROP TABLE IF EXISTS TEST1; +DROP TABLE IF EXISTS TEST2; +CREATE TABLE TEST1 (value Float64) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE TEST2 (value String) ENGINE=MergeTree ORDER BY tuple(); +INSERT INTO TEST1 VALUES +(101), +(102); +INSERT INTO TEST2 FORMAT CSV +1 +2; + +INSERT INTO TEST2 FORMAT CSV +3 +4 + +6 +EOF + +$CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" 2>&1 | grep -o 'Syntax error' + +# insert 7, 8, 9 into test2, because we use semicolon to determine the end of insert query(format is VALUES) +# then select all data from test1 and test2 +cat << EOF > "$SQL_FILE_NAME" +INSERT INTO TEST2 VALUES +('7'), +('8'), + +('9'); + +SELECT * FROM TEST1 ORDER BY value; +SELECT * FROM TEST2 ORDER BY value; +DROP TABLE TEST1; DROP TABLE TEST2; +EOF + +$CLICKHOUSE_CLIENT -m -n < "$SQL_FILE_NAME" + +rm "$SQL_FILE_NAME" From 5b5e88b7e34567659a6df2d1fc7abec53ff5c3b2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 11:00:21 +0000 Subject: [PATCH 316/321] Ok --- .../queries/0_stateless/01238_http_memory_tracking.sh | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index 26d3dd8acd4..ce1310cf302 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -8,18 +8,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -o pipefail +MISTER_USER="test_01238_http_$RANDOM" +# Create another user to check limit on the amount of memory reliabily +${CLICKHOUSE_CLIENT} --format Null -q "CREATE USER $MISTER_USER" + # This is needed to keep at least one running query for user for the time of test. # (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) -${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5000000 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & +${CLICKHOUSE_CLIENT} --user ${MISTER_USER} --function_sleep_max_microseconds_per_block 5000000 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { head -n1000 } | { - xargs -I{} ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((1<<30))" -d '{}' + xargs -I{} ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${MISTER_USER}&wait_end_of_query=1&max_memory_usage_for_user=$((1<<30))" -d '{}' } | grep -x -c 1 wait # Reset max_memory_usage_for_user, so it will not affect other tests -${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null" +${CLICKHOUSE_CLIENT} --user ${MISTER_USER} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null" From a8f36ebded4f2d8919a7713047f675126e7d9131 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 29 Jul 2024 13:01:47 +0200 Subject: [PATCH 317/321] update wording of toInt8OrDefault --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 37541dd4818..2fb7fa448a2 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -236,7 +236,7 @@ toInt8OrNull('abc'): ᴺᵁᴸᴸ ## toInt8OrDefault -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toInt8`](#toint8), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** From ceb0f7fe0f09fbb15cd107ba3f832735afca7c71 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 29 Jul 2024 13:17:27 +0200 Subject: [PATCH 318/321] Fix typoo --- docs/en/sql-reference/functions/type-conversion-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index bb386092c0f..339060a1ac4 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -335,7 +335,7 @@ toUInt8OrNull('abc'): ᴺᵁᴸᴸ ## toUInt8OrDefault -Like [`toUInt8`](#touint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`UInt8`](../data-types/int-uint.md). If unsuccessful, returns the default type value. +Like [`toUInt8`](#touint8), this function converts an input value to a value of type [UInt8](../data-types/int-uint.md) but returns the default value in case of an error. **Syntax** @@ -589,7 +589,7 @@ Like [`toUInt16`](#touint16), this function converts an input value to a value o **Syntax** ```sql -toInt16OrDefault(expr, def) +toUInt16OrDefault(expr, def) ``` **Arguments** From 2908cd7284d5b94ca5babe73fbdb4f1f6d7bb2fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 29 Jul 2024 13:28:24 +0200 Subject: [PATCH 319/321] update toInt*OrDefault to show that the default argument is optional --- .../functions/type-conversion-functions.md | 42 +++++++++++-------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 2fb7fa448a2..7cc2c022143 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -237,17 +237,18 @@ toInt8OrNull('abc'): ᴺᵁᴸᴸ ## toInt8OrDefault Like [`toInt8`](#toint8), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt8OrDefault(expr, def) +toInt8OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -265,7 +266,7 @@ This is not considered an error. **Returned value** -- 8-bit integer value if successful, otherwise returns the default value. [Int8](../data-types/int-uint.md). +- 8-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int8](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -486,17 +487,18 @@ toInt16OrNull('abc'): ᴺᵁᴸᴸ ## toInt16OrDefault Like [`toInt16`](#toint16), this function converts an input value to a value of type [Int16](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt16OrDefault(expr, def) +toInt16OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int16](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int16` is unsuccessful. [Int16](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -514,7 +516,7 @@ This is not considered an error. **Returned value** -- 16-bit integer value if successful, otherwise returns the default value. [Int16](../data-types/int-uint.md). +- 16-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int16](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -734,17 +736,18 @@ toInt32OrNull('abc'): ᴺᵁᴸᴸ ## toInt32OrDefault Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int32](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt32OrDefault(expr, def) +toInt32OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -762,7 +765,7 @@ This is not considered an error. **Returned value** -- 32-bit integer value if successful, otherwise returns the default value. [Int32](../data-types/int-uint.md). +- 32-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int32](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -983,17 +986,18 @@ toInt64OrNull('abc'): ᴺᵁᴸᴸ ## toInt64OrDefault Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int64](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt64OrDefault(expr, def) +toInt64OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -1011,7 +1015,7 @@ This is not considered an error. **Returned value** -- 64-bit integer value if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). +- 64-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int64](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1231,17 +1235,18 @@ toInt128OrNull('abc'): ᴺᵁᴸᴸ ## toInt128OrDefault Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int128](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt128OrDefault(expr, def) +toInt128OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). Supported arguments: - (U)Int8/16/32/64/128/256. @@ -1259,7 +1264,7 @@ This is not considered an error. **Returned value** -- 128-bit integer value if successful, otherwise returns the default value. [Int128](../data-types/int-uint.md). +- 128-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int128](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1479,17 +1484,18 @@ toInt256OrNull('abc'): ᴺᵁᴸᴸ ## toInt256OrDefault Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int256](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toInt256OrDefault(expr, def) +toInt256OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -1507,7 +1513,7 @@ This is not considered an error. **Returned value** -- 256-bit integer value if successful, otherwise returns the default value. [Int256](../data-types/int-uint.md). +- 256-bit integer value if successful, otherwise returns the default value if passed or `0` if not. [Int256](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. From 918feaff9dfbae9a8be71ee3a4b69abfb844b52a Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 29 Jul 2024 13:35:18 +0200 Subject: [PATCH 320/321] CI: Fix reuse for release branches --- tests/ci/pr_info.py | 9 ++++++++- tests/ci/test_ci_config.py | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 2c8ada7b983..5c051b093e0 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,6 +2,7 @@ import json import logging import os +import re from typing import Dict, List, Set, Union from urllib.parse import quote @@ -328,7 +329,13 @@ class PRInfo: @property def is_release(self) -> bool: - return self.number == 0 and not self.is_merge_queue + return self.is_master or ( + self.is_push_event + and ( + bool(re.match(r"^2[1-9]\.[1-9][0-9]*$", self.head_ref)) + or bool(re.match(r"^release/2[1-9]\.[1-9][0-9]*$", self.head_ref)) + ) + ) @property def is_pr(self): diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index be540413b3c..f376a129e6f 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -478,6 +478,7 @@ class TestCIConfig(unittest.TestCase): pr_info = PRInfo(github_event=_TEST_EVENT_JSON) pr_info.event_type = EventType.PUSH pr_info.number = 0 + pr_info.head_ref = "24.12345" assert pr_info.is_release and not pr_info.is_merge_queue ci_cache = CIPY._configure_jobs( S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True From e13bc4e47aeb12584c13a7a0fa1e6aa5e59e9770 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 29 Jul 2024 14:57:30 +0200 Subject: [PATCH 321/321] fix formatting issues --- .../functions/type-conversion-functions.md | 50 +++++++++++-------- 1 file changed, 28 insertions(+), 22 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 339060a1ac4..5f569f1b5c0 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -313,7 +313,7 @@ Query: ``` sql SELECT - toUInt8OrNull('-8'), + toUInt8OrNull('8'), toUInt8OrNull('abc') FORMAT vertical; ``` @@ -323,7 +323,7 @@ Result: ```response Row 1: ────── -toUInt8OrNull('-8'): ᴺᵁᴸᴸ +toUInt8OrNull('8'): 8 toUInt8OrNull('abc'): ᴺᵁᴸᴸ ``` @@ -336,17 +336,18 @@ toUInt8OrNull('abc'): ᴺᵁᴸᴸ ## toUInt8OrDefault Like [`toUInt8`](#touint8), this function converts an input value to a value of type [UInt8](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt8OrDefault(expr, def) +toUInt8OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt8` is unsuccessful. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `UInt8` is unsuccessful. [UInt8](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -355,7 +356,7 @@ Supported arguments: Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. -- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8OrDefault('0xc0fe', CAST('-1', 'UInt8'));`. +- String representations of binary and hexadecimal values, e.g. `SELECT toUInt8OrDefault('0xc0fe', CAST('0', 'UInt8'));`. :::note If the input value cannot be represented within the bounds of [UInt8](../data-types/int-uint.md), overflow or underflow of the result occurs. @@ -364,7 +365,7 @@ This is not considered an error. **Returned value** -- 8-bit unsigned integer value if successful, otherwise returns the default value. [UInt8](../data-types/int-uint.md). +- 8-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt8](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -585,17 +586,18 @@ toUInt16OrNull('abc'): ᴺᵁᴸᴸ ## toUInt16OrDefault Like [`toUInt16`](#touint16), this function converts an input value to a value of type [UInt16](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt16OrDefault(expr, def) +toUInt16OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt16` is unsuccessful. [UInt16](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `UInt16` is unsuccessful. [UInt16](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -613,12 +615,12 @@ This is not considered an error. **Returned value** -- 16-bit unsigned integer value if successful, otherwise returns the default value. [UInt16](../data-types/int-uint.md). +- 16-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt16](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. - ::: +::: **Example** @@ -835,17 +837,18 @@ toUInt32OrNull('abc'): ᴺᵁᴸᴸ ## toUInt32OrDefault Like [`toUInt32`](#touint32), this function converts an input value to a value of type [UInt32](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt32OrDefault(expr, def) +toUInt32OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt32` is unsuccessful. [UInt32](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `UInt32` is unsuccessful. [UInt32](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -863,7 +866,7 @@ This is not considered an error. **Returned value** -- 32-bit unsigned integer value if successful, otherwise returns the default value. [UInt32](../data-types/int-uint.md). +- 32-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt32](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1084,17 +1087,18 @@ toUInt64OrNull('abc'): ᴺᵁᴸᴸ ## toUInt64OrDefault Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt64OrDefault(expr, def) +toUInt64OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt64` is unsuccessful. [UInt64](../data-types/int-uint.md). +- `defauult` (optional) — The default value to return if parsing to type `UInt64` is unsuccessful. [UInt64](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -1112,7 +1116,7 @@ This is not considered an error. **Returned value** -- 64-bit unsigned integer value if successful, otherwise returns the default value. [UInt64](../data-types/int-uint.md). +- 64-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt64](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1332,17 +1336,18 @@ toUInt128OrNull('abc'): ᴺᵁᴸᴸ ## toUInt128OrDefault Like [`toUInt128`](#toint128), this function converts an input value to a value of type [UInt128](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt128OrDefault(expr, def) +toUInt128OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt128` is unsuccessful. [UInt128](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `UInt128` is unsuccessful. [UInt128](../data-types/int-uint.md). Supported arguments: - (U)Int8/16/32/64/128/256. @@ -1360,7 +1365,7 @@ This is not considered an error. **Returned value** -- 128-bit unsigned integer value if successful, otherwise returns the default value. [UInt128](../data-types/int-uint.md). +- 128-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt128](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1580,17 +1585,18 @@ toUInt256OrNull('abc'): ᴺᵁᴸᴸ ## toUInt256OrDefault Like [`toUInt256`](#touint256), this function converts an input value to a value of type [UInt256](../data-types/int-uint.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. **Syntax** ```sql -toUInt256OrDefault(expr, def) +toUInt256OrDefault(expr[, default]) ``` **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -- `def` — The default value to return if parsing to type `UInt256` is unsuccessful. [UInt256](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `UInt256` is unsuccessful. [UInt256](../data-types/int-uint.md). Supported arguments: - Values of type (U)Int8/16/32/64/128/256. @@ -1608,7 +1614,7 @@ This is not considered an error. **Returned value** -- 256-bit unsigned integer value if successful, otherwise returns the default value. [UInt256](../data-types/int-uint.md). +- 256-bit unsigned integer value if successful, otherwise returns the default value if passed or `0` if not. [UInt256](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers.